[jira] [Commented] (CASSANDRA-19572) Test failure: org.apache.cassandra.db.ImportTest flakiness

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19572:
---

I think I am onto something. I check the logs from the build for plain 4.0 
(without patch in 19401) and the first one is very interesting 
(testImportCorruptWithCopying) (1)

It fails on:
{code:java}
junit.framework.AssertionFailedError: 
expected:<[/tmp/importtest7641524017208283450/cql_test_keyspace/table_15-005af720fd9511ee865eef8364010360]>
 but 
was:<[/tmp/importtest7641524017208283450/cql_test_keyspace/table_15-005af720fd9511ee865eef8364010360,
 
/tmp/importtest916153905487802965/cql_test_keyspace/table_15-005af720fd9511ee865eef8364010360]>
at 
org.apache.cassandra.db.ImportTest.testCorruptHelper(ImportTest.java:341)
at 
org.apache.cassandra.db.ImportTest.testImportCorruptWithCopying(ImportTest.java:384)
 {code}
That test is expecting only one directory of sstables to be imported to be 
failed and another it expects to be loaded just fine, but here we clearly see 
that it failed to import {_}both{_}. I was checking the raw logs and I was 
quite lucky to find it, it is in this one (2). Grep it on exactly this 
timestamp:
{code}
ERROR [main] 2024-04-18 15:04:49,454 SSTableImporter.java:102
{code}
There you see that it failed to import the directory which it is not supposed 
to, that is the first stacktrace, but below it, there is another one:
{code}
[junit-timeout] ERROR [main] 2024-04-18 15:04:49,469 SSTableImporter.java:147 - 
Failed importing sstables in directory 
/tmp/importtest916153905487802965/cql_test_keyspace/table_15-005af720fd9511ee865eef8364010360
[junit-timeout] java.lang.AssertionError: null
[junit-timeout] at 
org.apache.cassandra.utils.concurrent.Ref$State.assertNotReleased(Ref.java:196)
[junit-timeout] at 
org.apache.cassandra.utils.concurrent.Ref.ref(Ref.java:152)
[junit-timeout] at 
org.apache.cassandra.io.sstable.format.SSTableReader$GlobalTidy.get(SSTableReader.java:2196)
[junit-timeout] at 
org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier.setup(SSTableReader.java:2028)
[junit-timeout] at 
org.apache.cassandra.io.sstable.format.SSTableReader.setup(SSTableReader.java:1971)
[junit-timeout] at 
org.apache.cassandra.io.sstable.format.SSTableReaderBuilder$ForRead.build(SSTableReaderBuilder.java:370)
[junit-timeout] at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:501)
[junit-timeout] at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:372)
[junit-timeout] at 
org.apache.cassandra.db.SSTableImporter.getTargetDirectory(SSTableImporter.java:211)
[junit-timeout] at 
org.apache.cassandra.db.SSTableImporter.importNewSSTables(SSTableImporter.java:135)
[junit-timeout] at 
org.apache.cassandra.db.ImportTest.testCorruptHelper(ImportTest.java:340)
[junit-timeout] at 
org.apache.cassandra.db.ImportTest.testImportCorruptWithCopying(ImportTest.java:384)
[junit-timeout] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native 
Method)
[junit-timeout] at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
{code}

Here we see that it was asserting that sstable reader is not released but it 
failed because it seems it is. That release is happening here (3).

I run the multiplexer on this (4) test for 3000x (5) and it all passed. I think 
we should just call "SSTableReader.resetTidying();". That method is actually 
annotated with VisibleForTesting. I think that resetting the tidying will clear 
underlying map of references so it will not complain afterwards. It is probably 
some concurrent thing or similar ... 

(1) 
[https://app.circleci.com/pipelines/github/instaclustr/cassandra/4199/workflows/a70b41d8-f848-4114-9349-9a01ac082281/jobs/223621/tests]
(2) 
[https://circleci.com/api/v1.1/project/github/instaclustr/cassandra/223621/output/103/11?file=true=662134c47c6ecf4bb1db4681-11-build%2FABCDEFGH]
(3) 
[https://github.com/apache/cassandra/blob/cassandra-4.0/test/unit/org/apache/cassandra/db/ImportTest.java#L235]
(4) 
https://github.com/apache/cassandra/pull/3264/commits/d934e1c0f40353a12cd7588fc8a15a23d35d6a30
(5) 
https://app.circleci.com/pipelines/github/instaclustr/cassandra/4210/workflows/eea52e61-b670-4dc9-86b6-b07bf1030b09/jobs/224285

> Test failure: org.apache.cassandra.db.ImportTest flakiness
> --
>
> Key: CASSANDRA-19572
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19572
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/bulk load
>Reporter: Brandon Williams
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>
> As discovered on 

[jira] [Comment Edited] (CASSANDRA-19572) Test failure: org.apache.cassandra.db.ImportTest flakiness

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19572 at 4/22/24 2:07 PM:


[~brandon.williams] do you still consider this to be a blocker for 
CASSANDRA-19401 ? I do not know what to make of it after your last comment in 
this ticket.


was (Author: smiklosovic):
[~brandon.williams] do you still consider this to be a blocker for 
CASSANDRA-19401. I do not know what to make of it after your last comment in 
this ticket.

> Test failure: org.apache.cassandra.db.ImportTest flakiness
> --
>
> Key: CASSANDRA-19572
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19572
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/bulk load
>Reporter: Brandon Williams
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>
> As discovered on CASSANDRA-19401, the tests in this class are flaky, at least 
> the following:
>  * testImportCorruptWithoutValidationWithCopying
>  * testImportInvalidateCache
>  * testImportCorruptWithCopying
>  * testImportCacheEnabledWithoutSrcDir
>  * testImportInvalidateCache
> [https://app.circleci.com/pipelines/github/instaclustr/cassandra/4199/workflows/a70b41d8-f848-4114-9349-9a01ac082281/jobs/223621/tests]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19572) Test failure: org.apache.cassandra.db.ImportTest flakiness

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19572:
---

[~brandon.williams] do you still consider this to be a blocker for 
CASSANDRA-19401. I do not know what to make of it after your last comment in 
this ticket.

> Test failure: org.apache.cassandra.db.ImportTest flakiness
> --
>
> Key: CASSANDRA-19572
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19572
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/bulk load
>Reporter: Brandon Williams
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>
> As discovered on CASSANDRA-19401, the tests in this class are flaky, at least 
> the following:
>  * testImportCorruptWithoutValidationWithCopying
>  * testImportInvalidateCache
>  * testImportCorruptWithCopying
>  * testImportCacheEnabledWithoutSrcDir
>  * testImportInvalidateCache
> [https://app.circleci.com/pipelines/github/instaclustr/cassandra/4199/workflows/a70b41d8-f848-4114-9349-9a01ac082281/jobs/223621/tests]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19566:
--
Status: Needs Committer  (was: Patch Available)

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

configuration_test.TestConfiguration test_change_durable_writes is 
CASSANDRA-19465 which I see returning back recently, nothing to do with this 
patch though.

write_failures_test.TestWriteFailures test_mutation_v5 passes locally

[CASSANDRA-19566-trunk|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19566-trunk]
{noformat}
java17_pre-commit_tests 
  ✓ j17_build4m 43s
  ✓ j17_cqlsh_dtests_py311   6m 54s
  ✓ j17_cqlsh_dtests_py311_vnode 7m 36s
  ✓ j17_cqlsh_dtests_py387m 26s
  ✓ j17_cqlsh_dtests_py38_vnode  7m 30s
  ✓ j17_cqlshlib_cython_tests7m 43s
  ✓ j17_cqlshlib_tests   8m 45s
  ✓ j17_dtests  36m 55s
  ✓ j17_unit_tests  13m 17s
  ✓ j17_utests_latest   13m 54s
  ✕ j17_dtests_latest   36m 42s
  configuration_test.TestConfiguration test_change_durable_writes
  ✕ j17_dtests_vnode36m 23s
  write_failures_test.TestWriteFailures test_mutation_v5
  ✕ j17_jvm_dtests  30m 39s
  
org.apache.cassandra.distributed.test.NativeTransportEncryptionOptionsTest 
testOptionalMtlsModeDoNotAllowNonSSLConnections TIMEOUTED
  org.apache.cassandra.distributed.test.tcm.SplitBrainTest 
testSplitBrainStartup TIMEOUTED
  ✕ j17_jvm_dtests_latest_vnode 28m 39s
  junit.framework.TestSuite 
org.apache.cassandra.fuzz.harry.integration.model.InJVMTokenAwareExecutorTest 
TIMEOUTED
  org.apache.cassandra.distributed.test.tcm.SplitBrainTest 
testSplitBrainStartup TIMEOUTED
  ✕ j17_utests_oa   13m 38s
  org.apache.cassandra.tcm.DiscoverySimulationTest discoveryTest
{noformat}

[java17_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4209/workflows/c5bb9ae4-b124-412e-90ac-8bb2657d1a2c]


> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

As I am writing this, I am running trunk's pre-commit workflow in CircleCI. I 
do not have my own instance of Jenkins and I do not believe that, looking into 
the history, cassandra-5-devbrach here (1) will actually test anything in a 
reasonable time without failures to provide upgrade tests. E.g. the last build 
is stuck on summary for 15 hours? 

(1) [https://ci-cassandra.apache.org/view/patches/job/Cassandra-5-devbranch/]

I would be delighted if community participated in the builds and provided 
upgrade tests as requested, most ideally via CircleCI. 

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

[CASSANDRA-19566-4.1|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19566-4.1]
{noformat}
java11_pre-commit_tests 
  ✓ j11_build2m 20s
  ✓ j11_cqlsh_dtests_py3 5m 35s
  ✓ j11_cqlsh_dtests_py311   6m 11s
  ✓ j11_cqlsh_dtests_py311_vnode 6m 25s
  ✓ j11_cqlsh_dtests_py385m 50s
  ✓ j11_cqlsh_dtests_py38_vnode  5m 55s
  ✓ j11_cqlsh_dtests_py3_vnode   5m 45s
  ✓ j11_cqlshlib_cython_tests7m 41s
  ✓ j11_cqlshlib_tests7m 1s
  ✓ j11_dtests  34m 43s
  ✓ j11_dtests_vnode36m 19s
  ✓ j11_jvm_dtests  19m 12s
  ✓ j11_jvm_dtests_vnode 12m 7s
  ✕ j11_unit_tests   8m 48s
  org.apache.cassandra.cql3.MemtableSizeTest testSize[skiplist]
{noformat}

[java11_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4206/workflows/5fb136cf-1503-40e0-a2a7-eeb93edf747b]


> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-22 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

[CASSANDRA-19566-4.0|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19566-4.0]
{noformat}
java8_pre-commit_tests  
  ✓ j8_build  5m 9s
  ✓ j8_cqlsh-dtests-py2-no-vnodes6m 30s
  ✓ j8_cqlsh-dtests-py2-with-vnodes  8m 15s
  ✓ j8_cqlsh_dtests_py3  7m 52s
  ✓ j8_cqlsh_dtests_py3117m 39s
  ✓ j8_cqlsh_dtests_py311_vnode  9m 13s
  ✓ j8_cqlsh_dtests_py38 7m 35s
  ✓ j8_cqlsh_dtests_py38_vnode   10m 6s
  ✓ j8_cqlsh_dtests_py3_vnode8m 31s
  ✓ j8_cqlshlib_tests8m 55s
  ✓ j8_dtests   32m 10s
  ✓ j8_dtests_vnode 36m 28s
  ✓ j8_jvm_dtests16m 6s
  ✓ j11_dtests_vnode36m 12s
  ✓ j11_dtests   34m 0s
  ✓ j11_cqlsh_dtests_py3_vnode   5m 46s
  ✓ j11_cqlsh_dtests_py38_vnode  5m 39s
  ✓ j11_cqlsh_dtests_py385m 58s
  ✓ j11_cqlsh_dtests_py311_vnode 5m 52s
  ✓ j11_cqlsh_dtests_py311   5m 24s
  ✓ j11_cqlsh_dtests_py3 5m 24s
  ✓ j11_cqlsh-dtests-py2-with-vnodes 5m 38s
  ✓ j11_cqlsh-dtests-py2-no-vnodes   5m 40s
  ✕ j8_unit_tests   10m 17s
  org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace
  ✕ j8_utests_system_keyspace_directory   8m 9s
  org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace
  ✕ j11_unit_tests   7m 58s
  org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace
  org.apache.cassandra.cql3.ViewFilteringClustering2Test 
testClusteringKeyFilteringRestrictions[0]
{noformat}

[java8_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4207/workflows/1b8ba460-20c2-42ad-8d89-704a09c4d211]


> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-19 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

I dont know how to run upgrades on my end. I dont have CircleCI plan which 
would not timeout for me and Jenkins still broken? 

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

Now we talk. I wonder how far this bug go. It is probably everywhere.

[https://app.circleci.com/pipelines/github/instaclustr/cassandra?branch=CASSANDRA-19566-5.0]

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19566 at 4/18/24 8:52 PM:


Now we talk. I wonder how far this bug go. It is probably everywhere.

[https://app.circleci.com/pipelines/github/instaclustr/cassandra/4200/workflows/1a1e48c3-f02a-45fd-8576-d4ae470a8199]


was (Author: smiklosovic):
Now we talk. I wonder how far this bug go. It is probably everywhere.

[https://app.circleci.com/pipelines/github/instaclustr/cassandra?branch=CASSANDRA-19566-5.0]

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19401:
--
Status: Ready to Commit  (was: Changes Suggested)

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

OK so given CASSANDRA-19572 I think we can merge this. Moving to ready to 
commit, correct me if not desirable. 

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19401:
--
Status: Changes Suggested  (was: Ready to Commit)

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19572) Test failure: org.apache.cassandra.db.ImportTest flakiness

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19572:
--
Description: 
As discovered on CASSANDRA-19401, the tests in this class are flaky, at least 
the following:
 * testImportCorruptWithoutValidationWithCopying
 * testImportInvalidateCache
 * testImportCorruptWithCopying
 * testImportCacheEnabledWithoutSrcDir
 * testImportInvalidateCache

[https://app.circleci.com/pipelines/github/instaclustr/cassandra/4199/workflows/a70b41d8-f848-4114-9349-9a01ac082281/jobs/223621/tests]

  was:
As discovered on CASSANDRA-19401, the tests in this class are flaky, at least 
the following:
 * testImportCorruptWithoutValidationWithCopying
 * testImportInvalidateCache
 * testImportCorruptWithCopying
 * testImportCacheEnabledWithoutSrcDir
 * testImportInvalidateCache

https://app.circleci.com/pipelines/github/instaclustr/cassandra/4199/workflows/a70b41d8-f848-4114-9349-9a01ac082281/jobs/223621/tests


> Test failure: org.apache.cassandra.db.ImportTest flakiness
> --
>
> Key: CASSANDRA-19572
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19572
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/bulk load
>Reporter: Brandon Williams
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>
> As discovered on CASSANDRA-19401, the tests in this class are flaky, at least 
> the following:
>  * testImportCorruptWithoutValidationWithCopying
>  * testImportInvalidateCache
>  * testImportCorruptWithCopying
>  * testImportCacheEnabledWithoutSrcDir
>  * testImportInvalidateCache
> [https://app.circleci.com/pipelines/github/instaclustr/cassandra/4199/workflows/a70b41d8-f848-4114-9349-9a01ac082281/jobs/223621/tests]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

We need to postpone the merging of this patch until we resolve the flakiness of 
ImportTest across all branches. 

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

Ah, so there are some problems in the end. I haven't looked in that yet.

[CASSANDRA-19566-5.0|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19566-5.0]
{noformat}
java17_pre-commit_tests 
  ✓ j17_build4m 36s
  ✓ j17_cqlsh_dtests_py311   5m 49s
  ✓ j17_cqlsh_dtests_py311_vnode 6m 39s
  ✓ j17_cqlsh_dtests_py38_vnode  6m 34s
  ✓ j17_cqlshlib_cython_tests7m 13s
  ✓ j17_cqlshlib_tests   8m 21s
  ✓ j17_dtests  32m 51s
  ✓ j17_dtests_latest   33m 11s
  ✓ j17_dtests_vnode33m 29s
  ✓ j17_jvm_dtests_latest_vnode 16m 56s
j17_jvm_dtests  22m 16s
j17_unit_tests   3m 26s
j17_cqlsh_dtests_py38   pending
j17_utests_latest   pending
  ✕ j17_utests_oa   15m 49s
  org.apache.cassandra.db.marshal.AbstractTypeTest serdeFromCQLLiteral
  org.apache.cassandra.db.marshal.AbstractTypeTest serdeFromString
  org.apache.cassandra.utils.bytecomparable.ByteSourceComparisonTest 
testDateType
  org.apache.cassandra.utils.bytecomparable.ByteSourceComparisonTest 
testTimestampType
  org.apache.cassandra.utils.bytecomparable.ByteSourceConversionTest 
testDateType
  org.apache.cassandra.utils.bytecomparable.ByteSourceConversionTest 
testTimestampType
  org.apache.cassandra.cql3.functions.CastFctsTest 
testTimeCastsInSelectionClause
  org.apache.cassandra.net.ConnectionTest testTimeout
  org.apache.cassandra.db.marshal.TimestampTypeTest stringProperty
java17_separate_tests
java11_pre-commit_tests 
java11_separate_tests
{noformat}

[java17_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4191/workflows/ed068a94-b146-4b8c-87fb-851be5735922]


> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19566 at 4/18/24 1:58 PM:


Ah, so there are some problems in the end. I haven't looked into that yet.

[CASSANDRA-19566-5.0|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19566-5.0]
{noformat}
java17_pre-commit_tests 
  ✓ j17_build4m 36s
  ✓ j17_cqlsh_dtests_py311   5m 49s
  ✓ j17_cqlsh_dtests_py311_vnode 6m 39s
  ✓ j17_cqlsh_dtests_py38_vnode  6m 34s
  ✓ j17_cqlshlib_cython_tests7m 13s
  ✓ j17_cqlshlib_tests   8m 21s
  ✓ j17_dtests  32m 51s
  ✓ j17_dtests_latest   33m 11s
  ✓ j17_dtests_vnode33m 29s
  ✓ j17_jvm_dtests_latest_vnode 16m 56s
j17_jvm_dtests  22m 16s
j17_unit_tests   3m 26s
j17_cqlsh_dtests_py38   pending
j17_utests_latest   pending
  ✕ j17_utests_oa   15m 49s
  org.apache.cassandra.db.marshal.AbstractTypeTest serdeFromCQLLiteral
  org.apache.cassandra.db.marshal.AbstractTypeTest serdeFromString
  org.apache.cassandra.utils.bytecomparable.ByteSourceComparisonTest 
testDateType
  org.apache.cassandra.utils.bytecomparable.ByteSourceComparisonTest 
testTimestampType
  org.apache.cassandra.utils.bytecomparable.ByteSourceConversionTest 
testDateType
  org.apache.cassandra.utils.bytecomparable.ByteSourceConversionTest 
testTimestampType
  org.apache.cassandra.cql3.functions.CastFctsTest 
testTimeCastsInSelectionClause
  org.apache.cassandra.net.ConnectionTest testTimeout
  org.apache.cassandra.db.marshal.TimestampTypeTest stringProperty
java17_separate_tests
java11_pre-commit_tests 
java11_separate_tests
{noformat}
[java17_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4191/workflows/ed068a94-b146-4b8c-87fb-851be5735922]


was (Author: smiklosovic):
Ah, so there are some problems in the end. I haven't looked in that yet.

[CASSANDRA-19566-5.0|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19566-5.0]
{noformat}
java17_pre-commit_tests 
  ✓ j17_build4m 36s
  ✓ j17_cqlsh_dtests_py311   5m 49s
  ✓ j17_cqlsh_dtests_py311_vnode 6m 39s
  ✓ j17_cqlsh_dtests_py38_vnode  6m 34s
  ✓ j17_cqlshlib_cython_tests7m 13s
  ✓ j17_cqlshlib_tests   8m 21s
  ✓ j17_dtests  32m 51s
  ✓ j17_dtests_latest   33m 11s
  ✓ j17_dtests_vnode33m 29s
  ✓ j17_jvm_dtests_latest_vnode 16m 56s
j17_jvm_dtests  22m 16s
j17_unit_tests   3m 26s
j17_cqlsh_dtests_py38   pending
j17_utests_latest   pending
  ✕ j17_utests_oa   15m 49s
  org.apache.cassandra.db.marshal.AbstractTypeTest serdeFromCQLLiteral
  org.apache.cassandra.db.marshal.AbstractTypeTest serdeFromString
  org.apache.cassandra.utils.bytecomparable.ByteSourceComparisonTest 
testDateType
  org.apache.cassandra.utils.bytecomparable.ByteSourceComparisonTest 
testTimestampType
  org.apache.cassandra.utils.bytecomparable.ByteSourceConversionTest 
testDateType
  org.apache.cassandra.utils.bytecomparable.ByteSourceConversionTest 
testTimestampType
  org.apache.cassandra.cql3.functions.CastFctsTest 
testTimeCastsInSelectionClause
  org.apache.cassandra.net.ConnectionTest testTimeout
  org.apache.cassandra.db.marshal.TimestampTypeTest stringProperty
java17_separate_tests
java11_pre-commit_tests 
java11_separate_tests
{noformat}

[java17_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4191/workflows/ed068a94-b146-4b8c-87fb-851be5735922]


> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, 

[jira] [Updated] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19401:
--
Status: Needs Committer  (was: Patch Available)

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

[CASSANDRA-19401-4.0|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19401-4.0]
{noformat}
java8_pre-commit_tests  
  ✓ j8_build 4m 54s
  ✓ j8_cqlsh-dtests-py2-no-vnodes6m 22s
  ✓ j8_cqlsh-dtests-py2-with-vnodes  8m 30s
  ✓ j8_cqlsh_dtests_py3  6m 22s
  ✓ j8_cqlsh_dtests_py311 9m 1s
  ✓ j8_cqlsh_dtests_py311_vnode  6m 20s
  ✓ j8_cqlsh_dtests_py38 8m 37s
  ✓ j8_cqlsh_dtests_py38_vnode7m 2s
  ✓ j8_cqlsh_dtests_py3_vnode5m 44s
  ✓ j8_cqlshlib_tests8m 13s
  ✓ j8_dtests   32m 11s
  ✓ j8_dtests_vnode 37m 10s
  ✓ j8_jvm_dtests   12m 46s
  ✓ j8_unit_tests_repeat 8m 12s
  ✓ j11_unit_tests_repeat7m 42s
  ✓ j11_dtests_vnode 35m 3s
  ✓ j11_dtests  31m 14s
  ✓ j11_cqlsh_dtests_py3_vnode   5m 29s
  ✓ j11_cqlsh_dtests_py38_vnode  5m 51s
  ✓ j11_cqlsh_dtests_py385m 26s
  ✓ j11_cqlsh_dtests_py311_vnode 5m 43s
  ✓ j11_cqlsh_dtests_py311   5m 56s
  ✓ j11_cqlsh_dtests_py3  6m 3s
  ✓ j11_cqlsh-dtests-py2-with-vnodes 5m 47s
  ✓ j11_cqlsh-dtests-py2-no-vnodes   5m 51s
  ✕ j8_unit_tests   10m 58s
  org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace
  ✕ j8_utests_system_keyspace_directory 10m 52s
  org.apache.cassandra.index.sasi.SASICQLTest testPagingWithClustering
  org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace
  ✕ j8_utests_system_keyspace_directory_repeat   8m 56s
  org.apache.cassandra.db.ImportTest testImportInvalidateCache
  ✕ j11_unit_tests   8m 13s
  org.apache.cassandra.net.ConnectionTest testMessageDeliveryOnReconnect
  org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace
{noformat}

[java8_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4197/workflows/fa55d5f0-4ed8-4125-bb9e-f83a22446050]

org.apache.cassandra.db.ImportTest testImportInvalidateCache seem to be a flaky 
one, similar case as for 5.0 build


> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import 

[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

working on it

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-18 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

[CASSANDRA-19401-trunk|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19401-trunk]
{noformat}
java11_pre-commit_tests 
  ✓ j11_build8m 35s
  ✓ j11_cqlsh_dtests_py311  10m 43s
  ✓ j11_cqlsh_dtests_py311_vnode 8m 33s
  ✓ j11_cqlsh_dtests_py388m 37s
  ✓ j11_cqlsh_dtests_py38_vnode  9m 16s
  ✓ j11_cqlshlib_cython_tests   12m 31s
  ✓ j11_cqlshlib_tests   10m 9s
  ✓ j11_jvm_dtests_latest_vnode 22m 51s
  ✓ j11_unit_tests  19m 22s
  ✓ j11_unit_tests_repeat   10m 45s
  ✓ j11_utests_latest17m 4s
  ✓ j11_utests_latest_repeat 9m 50s
  ✓ j11_utests_oa_repeat10m 37s
  ✓ j11_utests_system_keyspace_directory_repeat 10m 12s
  ✓ j17_cqlsh_dtests_py311   7m 19s
  ✓ j17_cqlsh_dtests_py311_vnode 7m 34s
  ✓ j17_cqlsh_dtests_py386m 55s
  ✓ j17_cqlsh_dtests_py38_vnode  7m 38s
  ✓ j17_cqlshlib_cython_tests7m 54s
  ✓ j17_cqlshlib_tests   6m 16s
  ✓ j17_unit_tests  13m 45s
  ✓ j17_unit_tests_repeat 6m 7s
  ✓ j17_utests_latest16m 6s
  ✓ j17_utests_latest_repeat 6m 42s
  ✓ j17_utests_oa   17m 23s
  ✓ j17_utests_oa_repeat  6m 6s
  ✕ j11_dtests  38m 56s
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j11_dtests_latest   37m 52s
  configuration_test.TestConfiguration test_change_durable_writes
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j11_dtests_vnode37m 21s
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j11_jvm_dtests  39m 56s
  org.apache.cassandra.distributed.test.log.FetchLogFromPeersTest 
testCMSCatchupTest TIMEOUTED
  ✕ j11_simulator_dtests36m 28s
  org.apache.cassandra.simulator.test.HarrySimulatorTest test
  org.apache.cassandra.simulator.test.ShortPaxosSimulationTest 
simulationTest
  ✕ j11_utests_oa   19m 44s
  org.apache.cassandra.net.ConnectionTest testTimeout
  ✕ j11_utests_system_keyspace_directory 21m 5s
  org.apache.cassandra.tools.TopPartitionsTest 
testServiceTopPartitionsSingleTable
  ✕ j17_dtests  37m 14s
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j17_dtests_latest   36m 41s
  configuration_test.TestConfiguration test_change_durable_writes
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j17_dtests_vnode 37m 1s
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j17_jvm_dtests  27m 32s
  
org.apache.cassandra.distributed.test.NativeTransportEncryptionOptionsTest 
testOptionalMtlsModeDoNotAllowNonSSLConnections TIMEOUTED
  org.apache.cassandra.distributed.test.log.FetchLogFromPeersTest 
testCMSCatchupTest
  org.apache.cassandra.distributed.test.tcm.SplitBrainTest 
testSplitBrainStartup TIMEOUTED
  ✕ j17_jvm_dtests_latest_vnode 22m 24s
  junit.framework.TestSuite 
org.apache.cassandra.fuzz.harry.integration.model.InJVMTokenAwareExecutorTest 
TIMEOUTED
java11_separate_tests
{noformat}

[java11_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4192/workflows/09240a96-74db-4549-bb0d-7ad9a6776b09]


> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table 

[jira] [Updated] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19401:
--
Fix Version/s: 4.0.x

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

for reviewers: I did it in such a way that only nodetool import will behave 
like it does not skip when the dir structure does not match. The listing is 
done elsewhere too and I have not touched that. It behaves as it was.

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

5.0 looks fine too. cqlsh_dtests failed on some circle / github networking 
issue.

ImportTest.testImportCorruptWithoutValidationWithCopying is just flaky and we 
discovered it here.

[CASSANDRA-19401-5.0|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19401-5.0]
{noformat}
java17_pre-commit_tests 
  ✓ j17_build3m 55s
  ✓ j17_cqlsh_dtests_py311   6m 18s
  ✓ j17_cqlsh_dtests_py311_vnode  6m 6s
  ✓ j17_cqlsh_dtests_py38 6m 9s
  ✓ j17_cqlshlib_cython_tests7m 47s
  ✓ j17_cqlshlib_tests   6m 34s
  ✓ j17_dtests  33m 27s
  ✓ j17_dtests_latest   32m 34s
  ✓ j17_dtests_vnode33m 32s
  ✓ j17_jvm_dtests  17m 57s
  ✓ j17_jvm_dtests_latest_vnode  16m 3s
  ✓ j17_unit_tests  16m 12s
  ✓ j17_utests_latest   14m 15s
  ✓ j17_utests_latest_repeat  6m 0s
  ✓ j17_utests_oa   14m 36s
  ✓ j17_utests_oa_repeat  6m 3s
  ✕ j17_cqlsh_dtests_py38_vnode  6m 31s
  ✕ j17_unit_tests_repeat   12m 43s
  org.apache.cassandra.db.ImportTest 
testImportCorruptWithoutValidationWithCopying
  org.apache.cassandra.db.ImportTest 
testImportCorruptWithoutValidationWithCopying
{noformat}

[java17_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4189/workflows/06a9fd1f-a7c6-4be4-bb45-ad09f1bc6e79]


> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To 

[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

It prints it via same means as a raw timestamp is. So if timestamp covers your 
edge case, tojson will too. If it does not, it wont.

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19566 at 4/17/24 1:53 PM:


It does not. I wonder what the reason might be.
{code:java}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}
Going into negative values before Jan 1 1970 works
{code:java}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=2;

 system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
+-+
  -1000 | 1969-12-31 23:59:59.00+ | "1969-12-31 
23:59:59.000Z"
{code}
but, obviously, for some big negative number is starts to break.

edit:

so, it starts to happen between
{code:java}
cqlsh:test> select id, tounixtimestamp(ts), ts, tojson(ts) from tbl where id in 
(13,16, 15, 17, 18);

 id | system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
++-+
 13 |-122000 | 1583-05-26 07:06:40.00+ | 
"1583-05-26 07:06:40.000Z"
 15 |-122250 | 1582-08-09 22:40:00.00+ | 
"1582-07-30 22:40:00.000Z"
 16 |-122150 | 1582-12-03 16:26:40.00+ | 
"1582-12-03 16:26:40.000Z"
 17 |-122200 | 1582-10-06 19:33:20.00+ | 
"1582-09-26 19:33:20.000Z"
 18 |-122180 | 1582-10-29 23:06:40.00+ | 
"1582-10-29 23:06:40.000Z"
{code}
29th October 1582 and 26th September 1582.

In 1582, there was Gregorian calendar reform introduced:

Date Adjustment: To realign the date of the vernal equinox to March 21st (which 
was the date of the equinox at the time of the First Council of Nicaea in AD 
325 and critical for determining the date of Easter), Thursday, October 4, 
1582, was followed by Friday, October 15, 1582. This correction effectively 
"skipped" 10 days in the calendar.

There is indeed drift of 10 days.

So what happens here is that when we go to negative values, whatever we use for 
date parsing does not correctly parse negative values when Gregorian calendar 
reform was introduced.


was (Author: smiklosovic):
It does not. I wonder what the reason might be.
{code:java}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}
Going into negative values before Jan 1 1970 works
{code:java}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=2;

 system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
+-+
  -1000 | 1969-12-31 23:59:59.00+ | "1969-12-31 
23:59:59.000Z"
{code}
but, obviously, for some big negative number is starts to break.

edit:

so, it starts to happen between
{code:java}
cqlsh:test> select id, tounixtimestamp(ts), ts, tojson(ts) from tbl where id in 
(13,16, 15, 17, 18);

 id | system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
++-+
 13 |-122000 | 1583-05-26 07:06:40.00+ | 
"1583-05-26 07:06:40.000Z"
 15 |-122250 | 1582-08-09 22:40:00.00+ | 
"1582-07-30 22:40:00.000Z"
 16 |-122150 | 1582-12-03 16:26:40.00+ | 
"1582-12-03 16:26:40.000Z"
 17 |-122200 | 1582-10-06 19:33:20.00+ | 
"1582-09-26 19:33:20.000Z"
 18 |-122180 | 1582-10-29 23:06:40.00+ | 
"1582-10-29 23:06:40.000Z"
{code}
29th October 1582 and 26th September 1582.

In 1582, there was Gregorian calendar introduced:

Date Adjustment: To realign the date of the vernal equinox to March 21st (which 
was the date of the equinox at the time of the First Council of Nicaea in AD 
325 and critical for determining the date of Easter), Thursday, October 4, 
1582, was followed by Friday, October 15, 1582. This 

[jira] [Updated] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19566:
--
Test and Documentation Plan: CI
 Status: Patch Available  (was: In Progress)

this solves it for 5.0

https://github.com/apache/cassandra/pull/3262

I ll run a build to be sure all plays fine.

{code}
cqlsh> select id, tounixtimestamp(ts), ts, tojson(ts) from test.tbl;

 id | system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
++-+
  5 |-125900 | 1571-01-15 09:46:40.00+ | 
"1571-01-15 09:46:40.000Z"
 10 |-125000 | 1573-11-22 01:46:40.00+ | 
"1573-11-22 01:46:40.000Z"
 16 |-122150 | 1582-12-03 16:26:40.00+ | 
"1582-12-03 16:26:40.000Z"
 13 |-122000 | 1583-05-26 07:06:40.00+ | 
"1583-05-26 07:06:40.000Z"
 11 |-124000 | 1577-01-22 11:33:20.00+ | 
"1577-01-22 11:33:20.000Z"
  1 |-125500 | 1572-04-22 08:53:20.00+ | 
"1572-04-22 08:53:20.000Z"
  8 |-125200 | 1573-04-04 14:13:20.00+ | 
"1573-04-04 14:13:20.000Z"
  2 |-125600 | 1571-12-28 15:06:40.00+ | 
"1571-12-28 15:06:40.000Z"
  4 |-125800 | 1571-05-11 03:33:20.00+ | 
"1571-05-11 03:33:20.000Z"
 18 |-122180 | 1582-10-29 23:06:40.00+ | 
"1582-10-29 23:06:40.000Z"
 15 |-122250 | 1582-08-09 22:40:00.00+ | 
"1582-08-09 22:40:00.000Z"
 20 | 1376701920 | 2406-04-05 12:00:00.00+ | 
"2406-04-05 12:00:00.000Z"
  7 |-125300 | 1572-12-09 20:26:40.00+ | 
"1572-12-09 20:26:40.000Z"
  6 |-125400 | 1572-08-16 02:40:00.00+ | 
"1572-08-16 02:40:00.000Z"
  9 |-125100 | 1573-07-29 08:00:00.00+ | 
"1573-07-29 08:00:00.000Z"
 14 |-122500 | 1581-10-24 14:13:20.00+ | 
"1581-10-24 14:13:20.000Z"
 17 |-122200 | 1582-10-06 19:33:20.00+ | 
"1582-10-06 19:33:20.000Z"
 12 |-123000 | 1580-03-24 21:20:00.00+ | 
"1580-03-24 21:20:00.000Z"
  3 |-125700 | 1571-09-03 21:20:00.00+ | 
"1571-09-03 21:20:00.000Z"

{code}

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

[jira] [Updated] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19566:
--
 Bug Category: Parent values: Correctness(12982)
   Complexity: Normal
  Component/s: Legacy/Core
   Legacy/CQL
Discovered By: Adhoc Test
Fix Version/s: 5.0.x
   5.x
 Severity: Low
 Assignee: Stefan Miklosovic
   Status: Open  (was: Triage Needed)

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core, Legacy/CQL
>Reporter: Bowen Song
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19566 at 4/17/24 1:46 PM:


It does not. I wonder what the reason might be.
{code:java}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}
Going into negative values before Jan 1 1970 works
{code:java}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=2;

 system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
+-+
  -1000 | 1969-12-31 23:59:59.00+ | "1969-12-31 
23:59:59.000Z"
{code}
but, obviously, for some big negative number is starts to break.

edit:

so, it starts to happen between
{code:java}
cqlsh:test> select id, tounixtimestamp(ts), ts, tojson(ts) from tbl where id in 
(13,16, 15, 17, 18);

 id | system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
++-+
 13 |-122000 | 1583-05-26 07:06:40.00+ | 
"1583-05-26 07:06:40.000Z"
 15 |-122250 | 1582-08-09 22:40:00.00+ | 
"1582-07-30 22:40:00.000Z"
 16 |-122150 | 1582-12-03 16:26:40.00+ | 
"1582-12-03 16:26:40.000Z"
 17 |-122200 | 1582-10-06 19:33:20.00+ | 
"1582-09-26 19:33:20.000Z"
 18 |-122180 | 1582-10-29 23:06:40.00+ | 
"1582-10-29 23:06:40.000Z"
{code}
29th October 1582 and 26th September 1582.

In 1582, there was Gregorian calendar introduced:

Date Adjustment: To realign the date of the vernal equinox to March 21st (which 
was the date of the equinox at the time of the First Council of Nicaea in AD 
325 and critical for determining the date of Easter), Thursday, October 4, 
1582, was followed by Friday, October 15, 1582. This correction effectively 
"skipped" 10 days in the calendar.

There is indeed drift of 10 days.

So what happens here is that when we go to negative values, whatever we use for 
date parsing does not correctly parse negative values when Gregorian calendar 
was introduced.


was (Author: smiklosovic):
It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

I wonder what sense it makes to enable timestamp to be a negative value.

Going into negative values before Jan 1 1970 works 

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=2;

 system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
+-+
  -1000 | 1969-12-31 23:59:59.00+ | "1969-12-31 
23:59:59.000Z"
{code}

but, obviously, for some big negative number is starts to break.

edit:

so, it starts to happen between

{code}
cqlsh:test> select id, tounixtimestamp(ts), ts, tojson(ts) from tbl where id in 
(13,16, 15, 17, 18);

 id | system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
++-+
 13 |-122000 | 1583-05-26 07:06:40.00+ | 
"1583-05-26 07:06:40.000Z"
 15 |-122250 | 1582-08-09 22:40:00.00+ | 
"1582-07-30 22:40:00.000Z"
 16 |-122150 | 1582-12-03 16:26:40.00+ | 
"1582-12-03 16:26:40.000Z"
 17 |-122200 | 1582-10-06 19:33:20.00+ | 
"1582-09-26 19:33:20.000Z"
 18 |-122180 | 1582-10-29 23:06:40.00+ | 
"1582-10-29 23:06:40.000Z"
{code}

29th October 1582 and 26th September 1582.

In 1582, there was Gregorian calendar introduced:

Date Adjustment: To realign the date of the vernal equinox to March 21st (which 
was the date of the equinox at the time of the First Council of Nicaea in AD 
325 and critical for determining the date of Easter), Thursday, October 4, 

[jira] [Comment Edited] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19566 at 4/17/24 12:46 PM:
-

It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

I wonder what sense it makes to enable timestamp to be a negative value.

Going into negative values before Jan 1 1970 works 

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=2;

 system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
+-+
  -1000 | 1969-12-31 23:59:59.00+ | "1969-12-31 
23:59:59.000Z"
{code}

but, obviously, for some big negative number is starts to break.

edit:

so, it starts to happen between

{code}
cqlsh:test> select id, tounixtimestamp(ts), ts, tojson(ts) from tbl where id in 
(13,16, 15, 17, 18);

 id | system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
++-+
 13 |-122000 | 1583-05-26 07:06:40.00+ | 
"1583-05-26 07:06:40.000Z"
 15 |-122250 | 1582-08-09 22:40:00.00+ | 
"1582-07-30 22:40:00.000Z"
 16 |-122150 | 1582-12-03 16:26:40.00+ | 
"1582-12-03 16:26:40.000Z"
 17 |-122200 | 1582-10-06 19:33:20.00+ | 
"1582-09-26 19:33:20.000Z"
 18 |-122180 | 1582-10-29 23:06:40.00+ | 
"1582-10-29 23:06:40.000Z"
{code}

29th October 1582 and 26th September 1582.

In 1582, there was Gregorian calendar introduced:

Date Adjustment: To realign the date of the vernal equinox to March 21st (which 
was the date of the equinox at the time of the First Council of Nicaea in AD 
325 and critical for determining the date of Easter), Thursday, October 4, 
1582, was followed by Friday, October 15, 1582. This correction effectively 
"skipped" 10 days in the calendar.

There is indeed drift of 10 days.

So what happens here is that when we go to negative values, whatever we use for 
date parsing does not correctly parse negative values when Gregorian calendar 
was introduced.


was (Author: smiklosovic):
It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

I wonder what sense it makes to enable timestamp to be a negative value.

Going into negative values before Jan 1 1970 works 

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=2;

 system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
+-+
  -1000 | 1969-12-31 23:59:59.00+ | "1969-12-31 
23:59:59.000Z"
{code}

but, obviously, for some big negative number is starts to break.

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Bowen Song
>Priority: Normal
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, 

[jira] [Comment Edited] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19566 at 4/17/24 12:28 PM:
-

It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

I wonder what sense it makes to enable timestamp to be a negative value.

Going into negative values before Jan 1 1970 works 

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=2;

 system.tounixtimestamp(ts) | ts  | 
system.tojson(ts)
+-+
  -1000 | 1969-12-31 23:59:59.00+ | "1969-12-31 
23:59:59.000Z"
{code}

but, obviously, for some big negative number is starts to break.


was (Author: smiklosovic):
It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

I wonder what sense it makes to enable timestamp to be a negative value.

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Bowen Song
>Priority: Normal
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Bowen Song
>Priority: Normal
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19566 at 4/17/24 12:24 PM:
-

It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

I wonder what sense it makes to enable timestamp to be a negative value.


was (Author: smiklosovic):
It does not. I wonder what the reason might be.

{code}
cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl;

 system.tounixtimestamp(ts) | ts                              | 
system.tojson(ts)
+-+
            -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
12:00:00.000Z"
             1376701920 | 2406-04-05 12:00:00.00+ | "2406-04-05 
12:00:00.000Z"
{code}

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Bowen Song
>Priority: Normal
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19566) JSON encoded timestamp value does not always match non-JSON encoded value

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19566:
---

Does it also happen when ts is not negative value?

> JSON encoded timestamp value does not always match non-JSON encoded value
> -
>
> Key: CASSANDRA-19566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19566
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Bowen Song
>Priority: Normal
>
> Description:
> "SELECT JSON ..." and "toJson(...)" on Cassandra 4.1.4 produces different 
> date than "SELECT ..."  for some timestamp type values.
>  
> Steps to reproduce:
> {code:java}
> $ sudo docker pull cassandra:4.1.4
> $ sudo docker create --name cass cassandra:4.1.4
> $ sudo docker start cass
> $ # wait for the Cassandra instance becomes ready
> $ sudo docker exec -ti cass cqlsh
> Connected to Test Cluster at 127.0.0.1:9042
> [cqlsh 6.1.0 | Cassandra 4.1.4 | CQL spec 3.4.6 | Native protocol v5]
> Use HELP for help.
> cqlsh> create keyspace test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> use test;
> cqlsh:test> create table tbl (id int, ts timestamp, primary key (id));
> cqlsh:test> insert into tbl (id, ts) values (1, -1376701920);
> cqlsh:test> select tounixtimestamp(ts), ts, tojson(ts) from tbl where id=1;
>  system.tounixtimestamp(ts) | ts                              | 
> system.tojson(ts)
> +-+
>             -1376701920 | 1533-09-28 12:00:00.00+ | "1533-09-18 
> 12:00:00.000Z"
> (1 rows)
> cqlsh:test> select json * from tbl where id=1;
>  [json]
> -
>  {"id": 1, "ts": "1533-09-18 12:00:00.000Z"}
> (1 rows)
> {code}
>  
> Expected behaviour:
> The "select ts", "select tojson(ts)" and "select json *" should all produce 
> the same date.
>  
> Actual behaviour:
> The "select ts" produced the "1533-09-28" date but the "select tojson(ts)" 
> and "select json *" produced the "1533-09-18" date.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

4.1 build seems reasonabley fine

[CASSANDRA-19401-4.1|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19401-4.1]
{noformat}
java11_pre-commit_tests 
  ✓ j11_build1m 28s
  ✓ j11_cqlsh_dtests_py3 5m 24s
  ✓ j11_cqlsh_dtests_py311   5m 25s
  ✓ j11_cqlsh_dtests_py311_vnode 6m 14s
  ✓ j11_cqlsh_dtests_py385m 47s
  ✓ j11_cqlsh_dtests_py38_vnode  5m 57s
  ✓ j11_cqlsh_dtests_py3_vnode   5m 33s
  ✓ j11_cqlshlib_cython_tests6m 55s
  ✓ j11_cqlshlib_tests6m 9s
  ✓ j11_dtests  33m 37s
  ✓ j11_jvm_dtests   19m 2s
  ✓ j11_jvm_dtests_vnode11m 38s
  ✓ j11_unit_tests_repeat 8m 4s
  ✕ j11_dtests_vnode33m 39s
  rebuild_test.TestRebuild test_simple_rebuild
  ✕ j11_unit_tests   7m 34s
  org.apache.cassandra.cql3.MemtableSizeTest testSize[skiplist]
{noformat}

[java11_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4187/workflows/9164b340-3472-4ba7-a1ae-6ab5cbf695cb]


> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19401 at 4/17/24 11:18 AM:
-

4.1 build seems reasonably fine

[CASSANDRA-19401-4.1|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19401-4.1]
{noformat}
java11_pre-commit_tests 
  ✓ j11_build1m 28s
  ✓ j11_cqlsh_dtests_py3 5m 24s
  ✓ j11_cqlsh_dtests_py311   5m 25s
  ✓ j11_cqlsh_dtests_py311_vnode 6m 14s
  ✓ j11_cqlsh_dtests_py385m 47s
  ✓ j11_cqlsh_dtests_py38_vnode  5m 57s
  ✓ j11_cqlsh_dtests_py3_vnode   5m 33s
  ✓ j11_cqlshlib_cython_tests6m 55s
  ✓ j11_cqlshlib_tests6m 9s
  ✓ j11_dtests  33m 37s
  ✓ j11_jvm_dtests   19m 2s
  ✓ j11_jvm_dtests_vnode11m 38s
  ✓ j11_unit_tests_repeat 8m 4s
  ✕ j11_dtests_vnode33m 39s
  rebuild_test.TestRebuild test_simple_rebuild
  ✕ j11_unit_tests   7m 34s
  org.apache.cassandra.cql3.MemtableSizeTest testSize[skiplist]
{noformat}
[java11_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4187/workflows/9164b340-3472-4ba7-a1ae-6ab5cbf695cb]


was (Author: smiklosovic):
4.1 build seems reasonabley fine

[CASSANDRA-19401-4.1|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19401-4.1]
{noformat}
java11_pre-commit_tests 
  ✓ j11_build1m 28s
  ✓ j11_cqlsh_dtests_py3 5m 24s
  ✓ j11_cqlsh_dtests_py311   5m 25s
  ✓ j11_cqlsh_dtests_py311_vnode 6m 14s
  ✓ j11_cqlsh_dtests_py385m 47s
  ✓ j11_cqlsh_dtests_py38_vnode  5m 57s
  ✓ j11_cqlsh_dtests_py3_vnode   5m 33s
  ✓ j11_cqlshlib_cython_tests6m 55s
  ✓ j11_cqlshlib_tests6m 9s
  ✓ j11_dtests  33m 37s
  ✓ j11_jvm_dtests   19m 2s
  ✓ j11_jvm_dtests_vnode11m 38s
  ✓ j11_unit_tests_repeat 8m 4s
  ✕ j11_dtests_vnode33m 39s
  rebuild_test.TestRebuild test_simple_rebuild
  ✕ j11_unit_tests   7m 34s
  org.apache.cassandra.cql3.MemtableSizeTest testSize[skiplist]
{noformat}

[java11_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4187/workflows/9164b340-3472-4ba7-a1ae-6ab5cbf695cb]


> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 1h
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> 

[jira] [Assigned] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic reassigned CASSANDRA-19401:
-

Assignee: Stefan Miklosovic

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19401:
---

[~nob13] this one is more complete patch 
[https://github.com/apache/cassandra/pull/3259]

would you verify that? I will create patches for other branches as well.

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19401) Nodetool import expects directory structure

2024-04-17 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19401:
--
Authors: Stefan Miklosovic
Test and Documentation Plan: CI
 Status: Patch Available  (was: In Progress)

> Nodetool import expects directory structure
> ---
>
> Key: CASSANDRA-19401
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19401
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Norbert Schultz
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> According to the 
> [documentation|https://cassandra.apache.org/doc/4.1/cassandra/operating/bulk_loading.html]
>  the nodetool import should not rely on the folder structure of the imported 
> sst files:
> {quote}
> Because the keyspace and table are specified on the command line for nodetool 
> import, there is not the same requirement as with sstableloader, to have the 
> SSTables in a specific directory path. When importing snapshots or 
> incremental backups with nodetool import, the SSTables don’t need to be 
> copied to another directory.
> {quote}
> However when importing old cassandra snapshots, we figured out, that sstables 
> still need to be in a directory called like $KEYSPACE/$TABLENAME files, even 
> when keyspace and table name are already present as parameters for the 
> nodetool import call.
> Call we used:
> {code}
> nodetool import --copy-data mykeyspace mytable /full_path_to/test1
> {code}
> Log:
> {code}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,565 
> SSTableImporter.java:72 - Loading new SSTables for mykeyspace/mytable: 
> Options{srcPaths='[/full_path_to/test1]', resetLevel=true, 
> clearRepaired=true, verifySSTables=true, verifyTokens=true, 
> invalidateCaches=true, extendedVerify=false, copyData= true}
> INFO  [RMI TCP Connection(21)-127.0.0.1] 2024-02-15 10:41:06,566 
> SSTableImporter.java:173 - No new SSTables were found for mykeyspace/mytable
> {code}
> However, when we move the sstables (.db-Files) to 
> {{alternative/mykeyspace/mytable}}
> and import with
> {code}
> nodetool import --copy-data mykeyspace mytable 
> /fullpath/alternative/mykeyspace/mytable
> {code}
> the import works
> {code}
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:177 - Loading new SSTables and building secondary 
> indexes for mykeyspace/mytable: 
> [BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-2-big-Data.db'),
>  
> BigTableReader(path='/mnt/ramdisk/cassandra4/data/mykeyspace/mytable-561a12d0cbe611eead78fbfd293cee40/me-1-big-Data.db')]
> INFO  [RMI TCP Connection(23)-127.0.0.1] 2024-02-15 10:43:36,093 
> SSTableImporter.java:190 - Done loading load new SSTables for 
> mykeyspace/mytable
> {code}
> We experienced this in Cassandra 4.1.3 on Java 11 (Linux)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19130) Implement transactional table truncation

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19130 at 4/16/24 8:38 PM:


[~aratnofsky]  feel free to expand that patch in the direction you fancy if you 
are interested in that. I am very open to the cooperation on this. 

 

?? TCM, then in compaction remove entire SSTables with maxDataAge <= 
truncatedAt, and remove all cells with earlier mutation timestamps. When we're 
handling a Truncation transformation in TCM (via TableTruncationListener), kick 
off a local truncation compaction??

isnt this quite a heavy operation? compacting as part of truncation? how would 
that look like when a node is restarted and truncations are replayed? You would 
compact on every such truncation record until all is replayed?

EDIT: I think that the logic would be same as now, basically ... it would 
truncate only if timestamp in TCM is bigger than what is persisted in system's 
truncatedAt field. 

Anyway, as I said, just show what you mean exactly, it is a little bit 
confusing for to me replicate what you mean.


was (Author: smiklosovic):
[~aratnofsky]  feel free to expand that patch in the direction you fancy if you 
are interested in that. I am very open to the cooperation on this. 

 

?? TCM, then in compaction remove entire SSTables with maxDataAge <= 
truncatedAt, and remove all cells with earlier mutation timestamps. When we're 
handling a Truncation transformation in TCM (via TableTruncationListener), kick 
off a local truncation compaction??

isnt this quite a heavy operation? compacting as part of truncation? how would 
that look like when a node is restarted and truncations are replayed? You would 
compact on every such truncation record until all is replayed?

 

> Implement transactional table truncation
> 
>
> Key: CASSANDRA-19130
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19130
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Consistency/Coordination
>Reporter: Marcus Eriksson
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> TRUNCATE table should leverage cluster metadata to ensure consistent 
> truncation timestamps across all replicas. The current implementation depends 
> on all nodes being available, but this could be reimplemented as a 
> {{Transformation}}.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19130) Implement transactional table truncation

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19130 at 4/16/24 8:33 PM:


[~aratnofsky]  feel free to expand that patch in the direction you fancy if you 
are interested in that. I am very open to the cooperation on this. 

 

?? TCM, then in compaction remove entire SSTables with maxDataAge <= 
truncatedAt, and remove all cells with earlier mutation timestamps. When we're 
handling a Truncation transformation in TCM (via TableTruncationListener), kick 
off a local truncation compaction??

isnt this quite a heavy operation? compacting as part of truncation? how would 
that look like when a node is restarted and truncations are replayed? You would 
compact on every such truncation record until all is replayed?

 


was (Author: smiklosovic):
[~aratnofsky]  feel free to expand that patch in the direction you fancy if you 
are interested in that. I am very open to the cooperation on this. 

> Implement transactional table truncation
> 
>
> Key: CASSANDRA-19130
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19130
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Consistency/Coordination
>Reporter: Marcus Eriksson
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> TRUNCATE table should leverage cluster metadata to ensure consistent 
> truncation timestamps across all replicas. The current implementation depends 
> on all nodes being available, but this could be reimplemented as a 
> {{Transformation}}.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19130) Implement transactional table truncation

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19130:
---

[~aratnofsky]  feel free to expand that patch in the direction you fancy if you 
are interested in that. I am very open to the cooperation on this. 

> Implement transactional table truncation
> 
>
> Key: CASSANDRA-19130
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19130
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Consistency/Coordination
>Reporter: Marcus Eriksson
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> TRUNCATE table should leverage cluster metadata to ensure consistent 
> truncation timestamps across all replicas. The current implementation depends 
> on all nodes being available, but this could be reimplemented as a 
> {{Transformation}}.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19553:
--
Status: Needs Committer  (was: Patch Available)

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Feature/Virtual Tables
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 'sai_sstable_indexes_per_query';
> {code}
> {code}
> cqlsh> select * from system_views.guardrails_threshold where name = 
> 'sai_sstable_indexes_per_query';
>  name  | fail | warn
> ---+--+--
>  sai_sstable_indexes_per_query |   20 |   16
> {code}
> {code}
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_warn_threshold from 32 to 16
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_fail_threshold from -1 to 20
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 10, fail = 5 where 
> name = 'sai_sstable_indexes_per_query';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="The 
> warn threshold 10 for sai_sstable_indexes_per_query_warn_threshold should be 
> lower than the fail threshold 5"
> {code}
> We can easily extend this to EnableFlag guardrails for which we would 
> dedicate a separate table (system_views.guadrails_flag).
> Other guardrail types are being investigated.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19429) Remove lock contention generated by getCapacity function in SSTableReader

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19429:
--
Status: Patch Available  (was: Review In Progress)

> Remove lock contention generated by getCapacity function in SSTableReader
> -
>
> Key: CASSANDRA-19429
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19429
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Dipietro Salvatore
>Assignee: Dipietro Salvatore
>Priority: Normal
> Fix For: 4.0.x, 4.1.x
>
> Attachments: Screenshot 2024-02-26 at 10.27.10.png, Screenshot 
> 2024-02-27 at 11.29.41.png, Screenshot 2024-03-19 at 15.22.50.png, 
> asprof_cass4.1.3__lock_20240216052912lock.html, 
> image-2024-03-08-15-51-30-439.png, image-2024-03-08-15-52-07-902.png
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Profiling Cassandra 4.1.3 on large AWS instances, a high number of lock 
> acquires is measured in the `getCapacity` function from 
> `org/apache/cassandra/cache/InstrumentingCache` (1.9M lock acquires per 60 
> seconds). Based on our tests on r8g.24xlarge instances (using Ubuntu 22.04), 
> this limits the CPU utilization of the system to under 50% when testing at 
> full load and therefore limits the achieved throughput.
> Removing the lock contention from the SSTableReader.java file by replacing 
> the call to `getCapacity` with `size` achieves up to 2.95x increase in 
> throughput on r8g.24xlarge and 2x on r7i.24xlarge:
> |Instance type|Cass 4.1.3|Cass 4.1.3 patched|
> |r8g.24xlarge|168k ops|496k ops (2.95x)|
> |r7i.24xlarge|153k ops|304k ops (1.98x)|
>  
> Instructions to reproduce:
> {code:java}
> ## Requirements for Ubuntu 22.04
> sudo apt install -y ant git openjdk-11-jdk
> ## Build and run
> CASSANDRA_USE_JDK11=true ant realclean && CASSANDRA_USE_JDK11=true ant jar && 
> CASSANDRA_USE_JDK11=true ant stress-build  && rm -rf data && bin/cassandra -f 
> -R
> # Run
> bin/cqlsh -e 'drop table if exists keyspace1.standard1;' && \
> bin/cqlsh -e 'drop keyspace if exists keyspace1;' && \
> bin/nodetool clearsnapshot --all && tools/bin/cassandra-stress write 
> n=1000 cl=ONE -rate threads=384 -node 127.0.0.1 -log file=cload.log 
> -graph file=cload.html && \
> bin/nodetool compact keyspace1   && sleep 30s && \
> tools/bin/cassandra-stress mixed ratio\(write=10,read=90\) duration=10m 
> cl=ONE -rate threads=406 -node localhost -log file=result.log -graph 
> file=graph.html
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19429) Remove lock contention generated by getCapacity function in SSTableReader

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19429:
---

I asked internally our techops to test this and we do not have such a big 
instances we run Cassandra on regularly so it is of a low interest to deal with 
this for now.

> Remove lock contention generated by getCapacity function in SSTableReader
> -
>
> Key: CASSANDRA-19429
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19429
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Dipietro Salvatore
>Assignee: Dipietro Salvatore
>Priority: Normal
> Fix For: 4.0.x, 4.1.x
>
> Attachments: Screenshot 2024-02-26 at 10.27.10.png, Screenshot 
> 2024-02-27 at 11.29.41.png, Screenshot 2024-03-19 at 15.22.50.png, 
> asprof_cass4.1.3__lock_20240216052912lock.html, 
> image-2024-03-08-15-51-30-439.png, image-2024-03-08-15-52-07-902.png
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Profiling Cassandra 4.1.3 on large AWS instances, a high number of lock 
> acquires is measured in the `getCapacity` function from 
> `org/apache/cassandra/cache/InstrumentingCache` (1.9M lock acquires per 60 
> seconds). Based on our tests on r8g.24xlarge instances (using Ubuntu 22.04), 
> this limits the CPU utilization of the system to under 50% when testing at 
> full load and therefore limits the achieved throughput.
> Removing the lock contention from the SSTableReader.java file by replacing 
> the call to `getCapacity` with `size` achieves up to 2.95x increase in 
> throughput on r8g.24xlarge and 2x on r7i.24xlarge:
> |Instance type|Cass 4.1.3|Cass 4.1.3 patched|
> |r8g.24xlarge|168k ops|496k ops (2.95x)|
> |r7i.24xlarge|153k ops|304k ops (1.98x)|
>  
> Instructions to reproduce:
> {code:java}
> ## Requirements for Ubuntu 22.04
> sudo apt install -y ant git openjdk-11-jdk
> ## Build and run
> CASSANDRA_USE_JDK11=true ant realclean && CASSANDRA_USE_JDK11=true ant jar && 
> CASSANDRA_USE_JDK11=true ant stress-build  && rm -rf data && bin/cassandra -f 
> -R
> # Run
> bin/cqlsh -e 'drop table if exists keyspace1.standard1;' && \
> bin/cqlsh -e 'drop keyspace if exists keyspace1;' && \
> bin/nodetool clearsnapshot --all && tools/bin/cassandra-stress write 
> n=1000 cl=ONE -rate threads=384 -node 127.0.0.1 -log file=cload.log 
> -graph file=cload.html && \
> bin/nodetool compact keyspace1   && sleep 30s && \
> tools/bin/cassandra-stress mixed ratio\(write=10,read=90\) duration=10m 
> cl=ONE -rate threads=406 -node localhost -log file=result.log -graph 
> file=graph.html
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19532:
--
  Fix Version/s: 5.1-alpha1
 (was: 5.x)
Source Control Link: 
https://github.com/apache/cassandra/commit/8d705b31e9a3f213e5eb296d4308e5b22b0e996c
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
> Fix For: 5.1-alpha1
>
> Attachments: ci_summary-1.html
>
>  Time Spent: 2h
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19560) Implement support for virtual types

2024-04-16 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19560:
--
Change Category: Operability
 Complexity: Normal
  Fix Version/s: 5.x
 Status: Open  (was: Triage Needed)

> Implement support for virtual types
> ---
>
> Key: CASSANDRA-19560
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19560
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Feature/Virtual Tables
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>
> We can not use user types in virtual tables. While it might look strange at 
> first to ask for the support of this, currently we can not do something like 
> this:
> {code}
> cqlsh> select * from system_guardrails.thresholds ;
>  name  | value
> ---+--
>collection_size |   {warn: 0, fail: 0}
>  column_value_size | {warn: -1, fail: -1}
>  columns_per_table | {warn: -1, fail: -1}
> ...
> {code}
> {code}
> VIRTUAL TABLE system_guardrails.thresholds (
> name text PRIMARY KEY,
> value settings
> ) WITH comment = 'Guardrails configuration table for thresholds';
> {code}
> because "settings" is a UDT 
> {code}
> cqlsh> DESCRIBE type system_guardrails.settings ;
> CREATE TYPE system_guardrails.settings (
> warn bigint,
> fail bigint
> );
> {code}
> While this is not absolutely necessary to implement and it might be worked 
> around by a simple tuple, it feels sad that user experience suffers. 
> Supporting custom types for vtables is indeed possible so we should just do 
> that.
> There is additional work needed to do this, because virtual types are not 
> supported in python-driver, I had to do this:
> https://github.com/smiklosovic/python-driver/commit/14b236bb471bc4a7e251a9f6b694de7885b339de
> python-driver reads system_schema.types in order to show the correct output 
> in cqlsh, as it has not recognized virtual types, it was alwasy displaying 
> the results like
> {code}
> settings(warn=-1, fail -1)
> {code}
> because it could not evaluate it differently. 
> With the patch for python-driver, it will fetch it from 
> system_virtual_schema.types, where it is like:
> {code}
> cqlsh> select * from system_virtual_schema.types ;
>  keyspace_name | type_name | field_names  | field_types
> ---+---+--+--
>  system_guardrails |  settings | ['warn', 'fail'] | ['bigint', 'bigint']
> (1 rows)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-12937) Default setting (yaml) for SSTable compression

2024-04-15 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-12937:
---

1/2. I am not sure, you could easily test this, just set default compactions / 
compressions on each node and see if schemas are in an agreement. 
3/4. I think that, ideally, out of the box, it should be same everywhere and a 
user would need to make a conscious choice if he wanted to override that. 
Currently, nothing prevents the configuration to be diverged. It is not about 
whether we should enable it or not, it should be rather about being sure that 
we do not diverge when we do not know about that.
5. I am not sure, it would be interesting to go over yaml and indentify these 
which _have to_ to be same everywhere, these would be candidates to put into 
TCM I guess.
6. currently we do not, I think that TCM would achieve that.
7. I think this all leads to some kind of an extension of nodetool which would 
talk to TCM and it would set defaults there which are not settable otherwise. 
E.g. default compression, so you would use a nodetool's subcommand to talk to 
TCM where you would say "commit this default compression for me", and then all 
nodes would apply these defaults from the log, eventually. Any other config 
change similar to this would be like that.

> Default setting (yaml) for SSTable compression
> --
>
> Key: CASSANDRA-12937
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12937
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Config
>Reporter: Michael Semb Wever
>Assignee: Stefan Miklosovic
>Priority: Low
>  Labels: AdventCalendar2021
> Fix For: 5.x
>
>  Time Spent: 8h
>  Remaining Estimate: 0h
>
> In many situations the choice of compression for sstables is more relevant to 
> the disks attached than to the schema and data.
> This issue is to add to cassandra.yaml a default value for sstable 
> compression that new tables will inherit (instead of the defaults found in 
> {{CompressionParams.DEFAULT}}.
> Examples where this can be relevant are filesystems that do on-the-fly 
> compression (btrfs, zfs) or specific disk configurations or even specific C* 
> versions (see CASSANDRA-10995 ).
> +Additional information for newcomers+
> Some new fields need to be added to {{cassandra.yaml}} to allow specifying 
> the field required for defining the default compression parameters. In 
> {{DatabaseDescriptor}} a new {{CompressionParams}} field should be added for 
> the default compression. This field should be initialized in 
> {{DatabaseDescriptor.applySimpleConfig()}}. At the different places where 
> {{CompressionParams.DEFAULT}} was used the code should call 
> {{DatabaseDescriptor#getDefaultCompressionParams}} that should return some 
> copy of configured {{CompressionParams}}.
> Some unit test using {{OverrideConfigurationLoader}} should be used to test 
> that the table schema use the new default when a new table is created (see 
> CreateTest for some example).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-12937) Default setting (yaml) for SSTable compression

2024-04-15 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-12937 at 4/15/24 12:06 PM:
-

{quote}It seems like if we actually want these to be cluster wide values and 
not configurable on a per-node basis the defaults themselves should be in 
TCM{quote}

Yes, I think this is the most ideal solution. If somebody wants to experiment 
with a new compressor and similar, there would need to be some knob to override 
it, like some JMX method or similar, and all risks attached to that (divergence 
of the configuration caused by operator's negligence) would be on him. 

However, who would be changing the defaults? What I mean by that is that if 
defaults are committed in TCM, then if we change our mind about the defaults, 
by what mean would we commit them into TCM again, now changed? 


was (Author: smiklosovic):
{quote}It seems like if we actually want these to be cluster wide values and 
not configurable on a per-node basis the defaults themselves should be in 
TCM{quote}

Yes, I think this is the most ideal solution. If somebody wants to experiment 
with a new compressor and similar, there would need to be some knob to override 
it, like some JMX method or similar, and all risks attached to that (divergence 
of the configuration caused by operator's negligence) would be on him. 

> Default setting (yaml) for SSTable compression
> --
>
> Key: CASSANDRA-12937
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12937
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Config
>Reporter: Michael Semb Wever
>Assignee: Stefan Miklosovic
>Priority: Low
>  Labels: AdventCalendar2021
> Fix For: 5.x
>
>  Time Spent: 8h
>  Remaining Estimate: 0h
>
> In many situations the choice of compression for sstables is more relevant to 
> the disks attached than to the schema and data.
> This issue is to add to cassandra.yaml a default value for sstable 
> compression that new tables will inherit (instead of the defaults found in 
> {{CompressionParams.DEFAULT}}.
> Examples where this can be relevant are filesystems that do on-the-fly 
> compression (btrfs, zfs) or specific disk configurations or even specific C* 
> versions (see CASSANDRA-10995 ).
> +Additional information for newcomers+
> Some new fields need to be added to {{cassandra.yaml}} to allow specifying 
> the field required for defining the default compression parameters. In 
> {{DatabaseDescriptor}} a new {{CompressionParams}} field should be added for 
> the default compression. This field should be initialized in 
> {{DatabaseDescriptor.applySimpleConfig()}}. At the different places where 
> {{CompressionParams.DEFAULT}} was used the code should call 
> {{DatabaseDescriptor#getDefaultCompressionParams}} that should return some 
> copy of configured {{CompressionParams}}.
> Some unit test using {{OverrideConfigurationLoader}} should be used to test 
> that the table schema use the new default when a new table is created (see 
> CreateTest for some example).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-12937) Default setting (yaml) for SSTable compression

2024-04-15 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-12937:
---

{quote}It seems like if we actually want these to be cluster wide values and 
not configurable on a per-node basis the defaults themselves should be in 
TCM{quote}

Yes, I think this is the most ideal solution. If somebody wants to experiment 
with a new compressor and similar, there would need to be some knob to override 
it, like some JMX method or similar, and all risks attached to that (divergence 
of the configuration caused by operator's negligence) would be on him. 

> Default setting (yaml) for SSTable compression
> --
>
> Key: CASSANDRA-12937
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12937
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Config
>Reporter: Michael Semb Wever
>Assignee: Stefan Miklosovic
>Priority: Low
>  Labels: AdventCalendar2021
> Fix For: 5.x
>
>  Time Spent: 8h
>  Remaining Estimate: 0h
>
> In many situations the choice of compression for sstables is more relevant to 
> the disks attached than to the schema and data.
> This issue is to add to cassandra.yaml a default value for sstable 
> compression that new tables will inherit (instead of the defaults found in 
> {{CompressionParams.DEFAULT}}.
> Examples where this can be relevant are filesystems that do on-the-fly 
> compression (btrfs, zfs) or specific disk configurations or even specific C* 
> versions (see CASSANDRA-10995 ).
> +Additional information for newcomers+
> Some new fields need to be added to {{cassandra.yaml}} to allow specifying 
> the field required for defining the default compression parameters. In 
> {{DatabaseDescriptor}} a new {{CompressionParams}} field should be added for 
> the default compression. This field should be initialized in 
> {{DatabaseDescriptor.applySimpleConfig()}}. At the different places where 
> {{CompressionParams.DEFAULT}} was used the code should call 
> {{DatabaseDescriptor#getDefaultCompressionParams}} that should return some 
> copy of configured {{CompressionParams}}.
> Some unit test using {{OverrideConfigurationLoader}} should be used to test 
> that the table schema use the new default when a new table is created (see 
> CreateTest for some example).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-12937) Default setting (yaml) for SSTable compression

2024-04-15 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-12937:
---

I don't think that nodes should diverge in any way when it comes to schemas. 
All the schema should be taken from TCM, irrelevant how local node is 
configured. I just do not see why would one want one node to compress with 
Deflate and another one to compress with lz4 ... Basically, it should ignore 
what is configured locally if it is different in TCM. 

This problem will probably be visible e.g. in Guardrails in TCM. What 
guardrails it should take into account when we join a new node and local 
guardrails configuration is different from the one in TCM? I would say that TCM 
should win here. 

> Default setting (yaml) for SSTable compression
> --
>
> Key: CASSANDRA-12937
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12937
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Config
>Reporter: Michael Semb Wever
>Assignee: Stefan Miklosovic
>Priority: Low
>  Labels: AdventCalendar2021
> Fix For: 5.x
>
>  Time Spent: 8h
>  Remaining Estimate: 0h
>
> In many situations the choice of compression for sstables is more relevant to 
> the disks attached than to the schema and data.
> This issue is to add to cassandra.yaml a default value for sstable 
> compression that new tables will inherit (instead of the defaults found in 
> {{CompressionParams.DEFAULT}}.
> Examples where this can be relevant are filesystems that do on-the-fly 
> compression (btrfs, zfs) or specific disk configurations or even specific C* 
> versions (see CASSANDRA-10995 ).
> +Additional information for newcomers+
> Some new fields need to be added to {{cassandra.yaml}} to allow specifying 
> the field required for defining the default compression parameters. In 
> {{DatabaseDescriptor}} a new {{CompressionParams}} field should be added for 
> the default compression. This field should be initialized in 
> {{DatabaseDescriptor.applySimpleConfig()}}. At the different places where 
> {{CompressionParams.DEFAULT}} was used the code should call 
> {{DatabaseDescriptor#getDefaultCompressionParams}} that should return some 
> copy of configured {{CompressionParams}}.
> Some unit test using {{OverrideConfigurationLoader}} should be used to test 
> that the table schema use the new default when a new table is created (see 
> CreateTest for some example).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19498) Error reading data from credential file

2024-04-15 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19498:
---

[~bschoeni] are you available to prepare 4.1 and 5.0 branches, please?

> Error reading data from credential file
> ---
>
> Key: CASSANDRA-19498
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19498
> Project: Cassandra
>  Issue Type: Bug
>  Components: Documentation, Tool/cqlsh
>Reporter: Slava
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The pylib/cqlshlib/cqlshmain.py code reads data from the credentials file, 
> however, it is immediately ignored.
> https://github.com/apache/cassandra/blob/c9625e0102dab66f41d3ef2338c54d499e73a8c5/pylib/cqlshlib/cqlshmain.py#L2070
> {code:java}
>     if not options.username:
>         credentials = configparser.ConfigParser()
>         if options.credentials is not None:
>             credentials.read(options.credentials)        # use the username 
> from credentials file but fallback to cqlshrc if username is absent from the 
> command line parameters
>         options.username = username_from_cqlshrc    if not options.password:
>         rawcredentials = configparser.RawConfigParser()
>         if options.credentials is not None:
>             rawcredentials.read(options.credentials)        # handling 
> password in the same way as username, priority cli > credentials > cqlshrc
>         options.password = option_with_default(rawcredentials.get, 
> 'plain_text_auth', 'password', password_from_cqlshrc)
>         options.password = password_from_cqlshrc{code}
> These corrections have been made in accordance with 
> https://issues.apache.org/jira/browse/CASSANDRA-16983 and 
> https://issues.apache.org/jira/browse/CASSANDRA-16456.
> The documentation does not indicate that AuthProviders can be used in the 
> cqlshrc and credentials files.
> I propose to return the ability to use the legacy option of specifying the 
> user and password in the credentials file in the [plain_text_auth] section.
> It is also required to describe the rules for using the credentials file in 
> the documentation.
> I can make a corresponding pull request.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19429) Remove lock contention generated by getCapacity function in SSTableReader

2024-04-12 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19429:
---

Sorry for the delay, I still plan to take a look. 

> Remove lock contention generated by getCapacity function in SSTableReader
> -
>
> Key: CASSANDRA-19429
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19429
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Dipietro Salvatore
>Assignee: Dipietro Salvatore
>Priority: Normal
> Fix For: 4.0.x, 4.1.x
>
> Attachments: Screenshot 2024-02-26 at 10.27.10.png, Screenshot 
> 2024-02-27 at 11.29.41.png, Screenshot 2024-03-19 at 15.22.50.png, 
> asprof_cass4.1.3__lock_20240216052912lock.html, 
> image-2024-03-08-15-51-30-439.png, image-2024-03-08-15-52-07-902.png
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Profiling Cassandra 4.1.3 on large AWS instances, a high number of lock 
> acquires is measured in the `getCapacity` function from 
> `org/apache/cassandra/cache/InstrumentingCache` (1.9M lock acquires per 60 
> seconds). Based on our tests on r8g.24xlarge instances (using Ubuntu 22.04), 
> this limits the CPU utilization of the system to under 50% when testing at 
> full load and therefore limits the achieved throughput.
> Removing the lock contention from the SSTableReader.java file by replacing 
> the call to `getCapacity` with `size` achieves up to 2.95x increase in 
> throughput on r8g.24xlarge and 2x on r7i.24xlarge:
> |Instance type|Cass 4.1.3|Cass 4.1.3 patched|
> |r8g.24xlarge|168k ops|496k ops (2.95x)|
> |r7i.24xlarge|153k ops|304k ops (1.98x)|
>  
> Instructions to reproduce:
> {code:java}
> ## Requirements for Ubuntu 22.04
> sudo apt install -y ant git openjdk-11-jdk
> ## Build and run
> CASSANDRA_USE_JDK11=true ant realclean && CASSANDRA_USE_JDK11=true ant jar && 
> CASSANDRA_USE_JDK11=true ant stress-build  && rm -rf data && bin/cassandra -f 
> -R
> # Run
> bin/cqlsh -e 'drop table if exists keyspace1.standard1;' && \
> bin/cqlsh -e 'drop keyspace if exists keyspace1;' && \
> bin/nodetool clearsnapshot --all && tools/bin/cassandra-stress write 
> n=1000 cl=ONE -rate threads=384 -node 127.0.0.1 -log file=cload.log 
> -graph file=cload.html && \
> bin/nodetool compact keyspace1   && sleep 30s && \
> tools/bin/cassandra-stress mixed ratio\(write=10,read=90\) duration=10m 
> cl=ONE -rate threads=406 -node localhost -log file=result.log -graph 
> file=graph.html
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-12 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19553:
---

I think that tuple approach for thresholds vtable is actually better, because 
JMX method / config methods for thresholds accept always two parameters as well 
- warn and fail. If we were to set it independently, then it would violate how 
we do that on JMX level too, basically.  Hence I think that it is better if we 
just copy what is already in place and modeling this by a tuple seems to be 
pretty handy. That would not require to make apply method non final as well. 

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Feature/Virtual Tables
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 'sai_sstable_indexes_per_query';
> {code}
> {code}
> cqlsh> select * from system_views.guardrails_threshold where name = 
> 'sai_sstable_indexes_per_query';
>  name  | fail | warn
> ---+--+--
>  sai_sstable_indexes_per_query |   20 |   16
> {code}
> {code}
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_warn_threshold from 32 to 16
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_fail_threshold from -1 to 20
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 10, fail = 5 where 
> name = 'sai_sstable_indexes_per_query';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="The 
> warn threshold 10 for sai_sstable_indexes_per_query_warn_threshold should be 
> lower than the fail threshold 5"
> {code}
> We can easily extend this to EnableFlag guardrails for which we would 
> dedicate a separate table (system_views.guadrails_flag).
> Other guardrail types are being investigated.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19532:
--
Status: Ready to Commit  (was: Review In Progress)

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
> Attachments: ci_summary.html, result_details.tar.gz
>
>  Time Spent: 2h
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19532:
---

I don't see that anything which failed is related to this change. +1.

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
> Attachments: ci_summary.html, result_details.tar.gz
>
>  Time Spent: 2h
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19532:
--
Fix Version/s: 5.x

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
> Fix For: 5.x
>
> Attachments: ci_summary.html, result_details.tar.gz
>
>  Time Spent: 2h
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19552) Nodetool to get/set guardrails configurations

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19552:
---

We do. https://issues.apache.org/jira/browse/CASSANDRA-16806

Since 4.1-alpha1

That being said, I do not think that it will make it to 4.1 branch as that is 
just a bug fix one. As a new feature, our best shot is to do that for trunk 
only.

> Nodetool to get/set guardrails configurations
> -
>
> Key: CASSANDRA-19552
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19552
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Feature/Guardrails
>Reporter: Yuqi Yan
>Assignee: Yuqi Yan
>Priority: Normal
> Fix For: 4.1.x
>
>
> Currently guardrails are only configurable through JMX / cassandra.yaml
> This provides a nodetool command to interact with all the getters/setters for 
> guardrails.
>  
> 4.1 PR: [https://github.com/apache/cassandra/pull/3243]
> trunk PR: [https://github.com/apache/cassandra/pull/3244]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19553 at 4/11/24 5:01 PM:


I'll park this ticket for a while as I am done (minus tests) and this might 
receive first round of reviews, I want to have an agreement on the approach.

for warning / failure thresholds, there are two columns, warn and fail, but it 
would be great if these thresholds are set together at once. When we update a 
column in a vtable, I get just that column, I do not have any visibility what 
other column is set to. So when I have "fail" set to 10 and "warn" set to 20, 
then this is a valid CQL but it does not make sense on guardrail level.

What I did in system_guardrails.thresholds table is that I made apply method 
non final to get all row to operate on.

I think this is a no-no and only other approach I can think of is to use a 
tuple like this:
{code:java}
cqlsh> select * from system_guardrails.thresholds_on_tuple ;
 name                          | value
---+--
               collection_size |   (0, 0)
             column_value_size | (-1, -1)
             columns_per_table | (-1, -1)
                fields_per_udt | (-1, -1)
   in_select_cartesian_product | (-1, -1)
          items_per_collection | (-1, -1)
                     keyspaces | (-1, -1)
         local_data_disk_usage | (-1, -1)
  materialized_views_per_table | (-1, -1)
    maximum_replication_factor | (-1, -1)
             maximum_timestamp |   (0, 0)
    minimum_replication_factor | (-1, -1)
             minimum_timestamp |   (0, 0)
                     page_size | (-1, -1)
      partition_keys_in_select | (-1, -1)
                partition_size |   (0, 0)
          partition_tombstones | (-1, -1)
 sai_sstable_indexes_per_query | (32, -1)
   secondary_indexes_per_table | (-1, -1)
                        tables | (-1, -1)
             vector_dimensions | (-1, -1) 

cqlsh> update system_guardrails.thresholds_on_tuple set value = (10, 20) where 
name = 'vector_dimensions';
cqlsh> select value from system_guardrails.thresholds_on_tuple where name = 
'vector_dimensions'; 

value
--
 (10, 20)
  {code}
Values guardrails look like this:
{code:java}
cqlsh> select * from system_guardrails.values ;

 name | disallowed | ignored | warned
--++-+
  read_consistency_levels |   {} |{} |   {}
 table_properties |   {} |{} |   {}
 write_consistency_levels |   {} |{} |   {}

(3 rows)
cqlsh> update system_guardrails.values set warned = {'EACH_QUORUM', 'ALL'} 
where name = 'read_consistency_levels';

(3 rows)
cqlsh> select warned from system_guardrails.values where name = 
'read_consistency_levels';

 warned

 {'ALL', 'EACH_QUORUM'}

(1 rows)
{code}
I think we are fully covered.


was (Author: smiklosovic):
I'll park this ticket for a while as I am done (minus tests) and this might 
receive first round of reviews, I want to have an agreement on the approach.

for warning / failure thresholds, there are two columns, warn and fail, but it 
would be great if these thresholds are set together at once. When we update a 
column in a vtable, I get just that column, I do not have any visibility what 
other column is set to. So when I have "fail" set to 10 and "warn" set to 20, 
then this is a valid CQL but it does not make sense on guardrail level.

What I did in system_guardrails.thresholds table is that I made apply method 
non final to get all row to operate on.

I think this is a no-no and only other approach I can think of is to use a 
tuple like this:
{code:java}
cqlsh> select * from system_guardrails.thresholds_on_tuple ;
 name                          | value
---+--
               collection_size |   (0, 0)
             column_value_size | (-1, -1)
             columns_per_table | (-1, -1)
                fields_per_udt | (-1, -1)
   in_select_cartesian_product | (-1, -1)
          items_per_collection | (-1, -1)
                     keyspaces | (-1, -1)
         local_data_disk_usage | (-1, -1)
  materialized_views_per_table | (-1, -1)
    maximum_replication_factor | (-1, -1)
             maximum_timestamp |   (0, 0)
    minimum_replication_factor | (-1, -1)
             minimum_timestamp |   (0, 0)
                     page_size | (-1, -1)
      partition_keys_in_select | (-1, -1)
                partition_size |   (0, 0)
          partition_tombstones | (-1, -1)
 sai_sstable_indexes_per_query | (32, -1)
   secondary_indexes_per_table | (-1, -1)
                        tables | (-1, -1)
             vector_dimensions | (-1, -1) 

cqlsh> update system_guardrails.thresholds_on_tuple set value = (10, 20) where 
name = 

[jira] [Updated] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19553:
--
Test and Documentation Plan: still need to finish test, I am just 
interested in the review for a general approach
 Status: Patch Available  (was: In Progress)

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Feature/Virtual Tables
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 'sai_sstable_indexes_per_query';
> {code}
> {code}
> cqlsh> select * from system_views.guardrails_threshold where name = 
> 'sai_sstable_indexes_per_query';
>  name  | fail | warn
> ---+--+--
>  sai_sstable_indexes_per_query |   20 |   16
> {code}
> {code}
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_warn_threshold from 32 to 16
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_fail_threshold from -1 to 20
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 10, fail = 5 where 
> name = 'sai_sstable_indexes_per_query';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="The 
> warn threshold 10 for sai_sstable_indexes_per_query_warn_threshold should be 
> lower than the fail threshold 5"
> {code}
> We can easily extend this to EnableFlag guardrails for which we would 
> dedicate a separate table (system_views.guadrails_flag).
> Other guardrail types are being investigated.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19553:
---

I'll park this ticket for a while as I am done (minus tests) and this might 
receive first round of reviews, I want to have an agreement on the approach.

for warning / failure thresholds, there are two columns, warn and fail, but it 
would be great if these thresholds are set together at once. When we update a 
column in a vtable, I get just that column, I do not have any visibility what 
other column is set to. So when I have "fail" set to 10 and "warn" set to 20, 
then this is a valid CQL but it does not make sense on guardrail level.

What I did in system_guardrails.thresholds table is that I made apply method 
non final to get all row to operate on.

I think this is a no-no and only other approach I can think of is to use a 
tuple like this:
{code:java}
cqlsh> select * from system_guardrails.thresholds_on_tuple ;
 name                          | value
---+--
               collection_size |   (0, 0)
             column_value_size | (-1, -1)
             columns_per_table | (-1, -1)
                fields_per_udt | (-1, -1)
   in_select_cartesian_product | (-1, -1)
          items_per_collection | (-1, -1)
                     keyspaces | (-1, -1)
         local_data_disk_usage | (-1, -1)
  materialized_views_per_table | (-1, -1)
    maximum_replication_factor | (-1, -1)
             maximum_timestamp |   (0, 0)
    minimum_replication_factor | (-1, -1)
             minimum_timestamp |   (0, 0)
                     page_size | (-1, -1)
      partition_keys_in_select | (-1, -1)
                partition_size |   (0, 0)
          partition_tombstones | (-1, -1)
 sai_sstable_indexes_per_query | (32, -1)
   secondary_indexes_per_table | (-1, -1)
                        tables | (-1, -1)
             vector_dimensions | (-1, -1) 

cqlsh> update system_guardrails.thresholds_on_tuple set value = (10, 20) where 
name = 'vector_dimensions';
cqlsh> select value from system_guardrails.thresholds_on_tuple where name = 
'vector_dimensions'; value
--
 (10, 20)
  {code}

Values guardrails look like this:

{code}
cqlsh> select * from system_guardrails.values ;

 name | disallowed | ignored | warned
--++-+
  read_consistency_levels |   {} |{} |   {}
 table_properties |   {} |{} |   {}
 write_consistency_levels |   {} |{} |   {}

(3 rows)
cqlsh> update system_guardrails.values set warned = {'EACH_QUORUM', 'ALL'} 
where name = 'read_consistency_levels';

(3 rows)
cqlsh> select warned from system_guardrails.values where name = 
'read_consistency_levels';

 warned

 {'ALL', 'EACH_QUORUM'}

(1 rows)
{code}

I think we are fully covered. 

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Feature/Virtual Tables
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 

[jira] [Commented] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19553:
---

for guardrail type of "Predicates", we have only replicaDiskUsage guardrail. 
What it does is that it guards if replicas for a write do not  have disk usages 
above some threshold and I do think this is applicable to be settable/gettable 
from a vtable.

 
We have PercentageThreshold of name localDataDiskUsage which is subtype of 
Threshold, that tracks warn and fail levels for a node a CQL query was executed 
against. That one we do show to a user and she can modify it.

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Feature/Virtual Tables
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 'sai_sstable_indexes_per_query';
> {code}
> {code}
> cqlsh> select * from system_views.guardrails_threshold where name = 
> 'sai_sstable_indexes_per_query';
>  name  | fail | warn
> ---+--+--
>  sai_sstable_indexes_per_query |   20 |   16
> {code}
> {code}
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_warn_threshold from 32 to 16
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_fail_threshold from -1 to 20
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 10, fail = 5 where 
> name = 'sai_sstable_indexes_per_query';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="The 
> warn threshold 10 for sai_sstable_indexes_per_query_warn_threshold should be 
> lower than the fail threshold 5"
> {code}
> We can easily extend this to EnableFlag guardrails for which we would 
> dedicate a separate table (system_views.guadrails_flag).
> Other guardrail types are being investigated.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19553:
--
Change Category: Operability
 Complexity: Normal
Component/s: Feature/Virtual Tables
  Fix Version/s: 5.x
 Status: Open  (was: Triage Needed)

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Feature/Virtual Tables
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 'sai_sstable_indexes_per_query';
> {code}
> {code}
> cqlsh> select * from system_views.guardrails_threshold where name = 
> 'sai_sstable_indexes_per_query';
>  name  | fail | warn
> ---+--+--
>  sai_sstable_indexes_per_query |   20 |   16
> {code}
> {code}
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_warn_threshold from 32 to 16
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_fail_threshold from -1 to 20
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 10, fail = 5 where 
> name = 'sai_sstable_indexes_per_query';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="The 
> warn threshold 10 for sai_sstable_indexes_per_query_warn_threshold should be 
> lower than the fail threshold 5"
> {code}
> We can easily extend this to EnableFlag guardrails for which we would 
> dedicate a separate table (system_views.guadrails_flag).
> Other guardrail types are being investigated.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19553:
---

Added enabled flags:

{code}
cqlsh> select * from system_views.guardrails_enable_flags ;

 name | value
--+---
  allow_filtering |  True
  alter_table |  True
bulk_load_enabled |  True
   compact_tables |  True
drop_keyspace_enabled |  True
  drop_truncate_table_enabled |  True
 group_by |  True
intersect_filtering_query |  True
 non_partition_restricted_index_query_enabled |  True
read_before_write_list_operations |  True
secondary_indexes |  True
   simplestrategy |  True
  uncompressed_tables_enabled | False
  user_timestamps |  True
 zero_ttl_on_twcs |  True

(15 rows)
cqlsh> update system_views.guardrails_enable_flags SET value = false where name 
= 'group_by';
{code}
{code}
INFO  [Native-Transport-Requests-1] 2024-04-11 13:15:51,861 
GuardrailsOptions.java:998 - Updated group_by_enabled from true to false
{code}
{code}
cqlsh> select value from system_views.guardrails_enable_flags where name = 
'group_by';

 value
---
 False

(1 rows)
{code}

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 'sai_sstable_indexes_per_query';
> {code}
> {code}
> cqlsh> select * from system_views.guardrails_threshold where name = 
> 'sai_sstable_indexes_per_query';
>  name  | fail | warn
> ---+--+--
>  sai_sstable_indexes_per_query |   20 |   16
> {code}
> {code}
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_warn_threshold from 32 to 16
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> sai_sstable_indexes_per_query_fail_threshold from -1 to 20
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 10, fail = 5 where 
> name = 'sai_sstable_indexes_per_query';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="The 
> warn threshold 10 for sai_sstable_indexes_per_query_warn_threshold should be 
> lower than the fail threshold 5"
> {code}
> We can easily extend this to EnableFlag guardrails for which we would 
> dedicate a separate table (system_views.guadrails_flag).
> Other guardrail types are being investigated.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19479) Fix type issues and provide tests for type compatibility between 4.1 and 5.0

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19479:
---

+1, I do not see anything suspicious in build's test results. 

> Fix type issues and provide tests for type compatibility between 4.1 and 5.0
> 
>
> Key: CASSANDRA-19479
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19479
> Project: Cassandra
>  Issue Type: Task
>  Components: Legacy/Core, Test/unit
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>
> This is a part of CASSANDRA-14476 - we should verify whether the type 
> compatibility matrix is upgradable from 4.0 and 4.1 to 5.0, and if not, fix 
> the remaining issues.
> The implemented tests verify the following:
> - assumed compatibility between primitive types
> - equals method symmetricity
> - freezing/unfreezing
> - value compatibility by using a serializer of one type to deserialize a 
> value serialized using a serializer of another type
> - serialization compatibility by serializing a row with a column of one type 
> as a column of another type for simple and complex cells (multicell types)
> - (comparison) compatibility by comparing serialized values of one type using 
> a comparator of another type; for multicell types - build rows and compare 
> cell paths of a complex type using a cell path comparator of another complex 
> type
> - verify whether types that are (value/serialization/comparison) compatible 
> in a previous release are still compatible with this release
> - store the compatibility matrix in a compressed JSON file so that we can 
> copy it to future releases to assert backward compatibility (similar approach 
> to LegacySSTableTest)
> - verify that type serializers are different for non-compatible type pairs 
> which use custom comparisons
> Additionally:
> - the equals method in {{TupleType}} and {{UserType}} was fixed to be 
> symmetric. Previously, comparing two values gave a different outcome when 
> inverted.
> - fixed a condition in comparison method of {{AbstractCompositeType}}
> - ported a fix for composite and dynamic composite types which adds a 
> distinct serializers for them so that the serializers for those types and for 
> {{BytesType}} are considered different; similar thing was done for 
> {{LexicalUUIDType}} to make its serializer different to {{UUIDType}} 
> serializer (see 
> https://the-asf.slack.com/archives/CK23JSY2K/p1712060572432959)
> - fixed a problem with DCT builder - in 5.0+ the {{DynamicCompositeType}} 
> generation has a problem with inverse alias-type mapping which makes it 
> vulnerable to problems when the same type has two different aliases



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19552) Nodetool to get/set guardrails configurations

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19552:
---

I created CASSANDRA-19553 for vtable approach.

> Nodetool to get/set guardrails configurations
> -
>
> Key: CASSANDRA-19552
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19552
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Feature/Guardrails
>Reporter: Yuqi Yan
>Assignee: Yuqi Yan
>Priority: Normal
> Fix For: 4.1.x
>
>
> Currently guardrails are only configurable through JMX / cassandra.yaml
> This provides a nodetool command to interact with all the getters/setters for 
> guardrails.
>  
> 4.1 PR: [https://github.com/apache/cassandra/pull/3243]
> trunk PR: [https://github.com/apache/cassandra/pull/3244]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19553:
--
Description: 
In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool and 
via reflection.

I think that it will be better if we do that via CQL.

So far, I have this:

{code}
cqlsh> select * from system_views.guardrails_threshold ;

 name  | fail  | warn
---+---+---
   collection_size | 0 | 0
 column_value_size |-1 |-1
 columns_per_table |-1 |-1
fields_per_udt |-1 |-1
   in_select_cartesian_product |-1 |-1
  items_per_collection |-1 |-1
 keyspaces |-1 |-1
  materialized_views_per_table |-1 |-1
maximum_replication_factor |-1 |-1
 maximum_timestamp | 0 | 0
minimum_replication_factor |-1 |-1
 minimum_timestamp | 0 | 0
 page_size |-1 |-1
  partition_keys_in_select |-1 |-1
partition_size | 40960 | 20480
  partition_tombstones |-1 |-1
 sai_sstable_indexes_per_query |-1 |32
   secondary_indexes_per_table |-1 |-1
tables |-1 |-1
 vector_dimensions |-1 |-1
{code}

{code}
cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 where 
name = 'sai_sstable_indexes_per_query';
{code}

{code}
cqlsh> select * from system_views.guardrails_threshold where name = 
'sai_sstable_indexes_per_query';

 name  | fail | warn
---+--+--
 sai_sstable_indexes_per_query |   20 |   16
{code}

{code}
INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
GuardrailsOptions.java:998 - Updated 
sai_sstable_indexes_per_query_warn_threshold from 32 to 16
INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
GuardrailsOptions.java:998 - Updated 
sai_sstable_indexes_per_query_fail_threshold from -1 to 20
{code}

{code}
cqlsh> update system_views.guardrails_threshold SET warn = 10, fail = 5 where 
name = 'sai_sstable_indexes_per_query';
InvalidRequest: Error from server: code=2200 [Invalid query] message="The warn 
threshold 10 for sai_sstable_indexes_per_query_warn_threshold should be lower 
than the fail threshold 5"
{code}

We can easily extend this to EnableFlag guardrails for which we would dedicate 
a separate table (system_views.guadrails_flag).

Other guardrail types are being investigated.

> get/set guardrails configuration via CQL
> 
>
> Key: CASSANDRA-19553
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
>
> In CASSANDRA-19552, there is a patch which modifies guardrails via nodetool 
> and via reflection.
> I think that it will be better if we do that via CQL.
> So far, I have this:
> {code}
> cqlsh> select * from system_views.guardrails_threshold ;
>  name  | fail  | warn
> ---+---+---
>collection_size | 0 | 0
>  column_value_size |-1 |-1
>  columns_per_table |-1 |-1
> fields_per_udt |-1 |-1
>in_select_cartesian_product |-1 |-1
>   items_per_collection |-1 |-1
>  keyspaces |-1 |-1
>   materialized_views_per_table |-1 |-1
> maximum_replication_factor |-1 |-1
>  maximum_timestamp | 0 | 0
> minimum_replication_factor |-1 |-1
>  minimum_timestamp | 0 | 0
>  page_size |-1 |-1
>   partition_keys_in_select |-1 |-1
> partition_size | 40960 | 20480
>   partition_tombstones |-1 |-1
>  sai_sstable_indexes_per_query |-1 |32
>secondary_indexes_per_table |-1 |-1
> tables |-1 |-1
>  vector_dimensions |-1 |-1
> {code}
> {code}
> cqlsh> update system_views.guardrails_threshold SET warn = 16, fail = 20 
> where name = 'sai_sstable_indexes_per_query';
> {code}
> {code}
> cqlsh> select * from system_views.guardrails_threshold where name = 
> 'sai_sstable_indexes_per_query';
>  name  | fail | warn
> ---+--+--
>  sai_sstable_indexes_per_query |   20 |   16
> {code}
> {code}
> INFO  [Native-Transport-Requests-1] 2024-04-11 11:51:24,483 
> GuardrailsOptions.java:998 - Updated 
> 

[jira] [Created] (CASSANDRA-19553) get/set guardrails configuration via CQL

2024-04-11 Thread Stefan Miklosovic (Jira)
Stefan Miklosovic created CASSANDRA-19553:
-

 Summary: get/set guardrails configuration via CQL
 Key: CASSANDRA-19553
 URL: https://issues.apache.org/jira/browse/CASSANDRA-19553
 Project: Cassandra
  Issue Type: New Feature
Reporter: Stefan Miklosovic
Assignee: Stefan Miklosovic






--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19552) Nodetool to get/set guardrails configurations

2024-04-11 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19552:
---

I am not sure we want to do this in the first place. There is a general trend 
moving away from nodetool towards CQL way of doing things so trying to do this 
by CQL would be probably more appropriate. 

> Nodetool to get/set guardrails configurations
> -
>
> Key: CASSANDRA-19552
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19552
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Feature/Guardrails
>Reporter: Yuqi Yan
>Assignee: Yuqi Yan
>Priority: Normal
> Fix For: 4.1.x
>
>
> Currently guardrails are only configurable through JMX / cassandra.yaml
> This provides a nodetool command to interact with all the getters/setters for 
> guardrails.
>  
> 4.1 PR: [https://github.com/apache/cassandra/pull/3243]
> trunk PR: [https://github.com/apache/cassandra/pull/3244]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-10 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19532:
--
Status: Review In Progress  (was: Changes Suggested)

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 2h
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19486) enrich system_views.pending_hints with hints sizes

2024-04-10 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19486:
--
  Fix Version/s: 5.1-alpha1
 (was: 5.x)
Source Control Link: 
https://github.com/apache/cassandra/commit/5b23692a909e1c2d5a7f8f5a17a791274221f363
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

> enrich system_views.pending_hints with hints sizes
> --
>
> Key: CASSANDRA-19486
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19486
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Consistency/Hints
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.1-alpha1
>
>  Time Spent: 1h 50m
>  Remaining Estimate: 0h
>
> I am happy to close this if somebody shows me how to get total size of all 
> hints or all hints per particular node via JMX.
> I could find StorageMetrics.totalHints but that is how many hints there are, 
> not their sizes. We also have
> org.apache.cassandra.metrics:type=HintedHandOffManager name=
> org.apache.cassandra.metrics:type=HintsService name=
> But that is again showing other metrics not sizes.
> I would add two methods into HintsServiceMBean returning this. Seems to be 
> very easy to do once we do CASSANDRA-19477.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19486) enrich system_views.pending_hints with hints sizes

2024-04-10 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19486:
--
Reviewers: Berenguer Blasi  (was: Berenguer Blasi, Maxim Muzafarov)

> enrich system_views.pending_hints with hints sizes
> --
>
> Key: CASSANDRA-19486
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19486
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Consistency/Hints
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> I am happy to close this if somebody shows me how to get total size of all 
> hints or all hints per particular node via JMX.
> I could find StorageMetrics.totalHints but that is how many hints there are, 
> not their sizes. We also have
> org.apache.cassandra.metrics:type=HintedHandOffManager name=
> org.apache.cassandra.metrics:type=HintsService name=
> But that is again showing other metrics not sizes.
> I would add two methods into HintsServiceMBean returning this. Seems to be 
> very easy to do once we do CASSANDRA-19477.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19486) enrich system_views.pending_hints with hints sizes

2024-04-10 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19486:
--
Status: Ready to Commit  (was: Review In Progress)

> enrich system_views.pending_hints with hints sizes
> --
>
> Key: CASSANDRA-19486
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19486
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Consistency/Hints
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> I am happy to close this if somebody shows me how to get total size of all 
> hints or all hints per particular node via JMX.
> I could find StorageMetrics.totalHints but that is how many hints there are, 
> not their sizes. We also have
> org.apache.cassandra.metrics:type=HintedHandOffManager name=
> org.apache.cassandra.metrics:type=HintsService name=
> But that is again showing other metrics not sizes.
> I would add two methods into HintsServiceMBean returning this. Seems to be 
> very easy to do once we do CASSANDRA-19477.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-10 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19532:
---

I was sleeping on it and I do not consider the second guardrail so important. 
It is sad that we can not have two proper guardrails because of TCM replay and 
config changes  so if we were left with the second guardrail only, looking 
into how similar features are turned on / off in cassandra.yaml, none of them 
are guardrail enabled. For example, user defined functions or materialized 
views or transient replication, dropping compact storage and similar. Also, I 
think that the probability of somebody wanting to dynamically enable / disable 
triggers in runtime is fairly minimal. I think that it is decided just once by 
whole devops / engineering team if triggers are going to be used for a specific 
deployment / project and these decisions are probably not changed frequently to 
justify added complexity. 

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19532:
--
Status: Review In Progress  (was: Patch Available)

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> shortly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19532:
--
Status: Changes Suggested  (was: Review In Progress)

commented few things on the PR

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> shortly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19486) enrich system_views.pending_hints with hints sizes

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19486:
---

[CASSANDRA-19486|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19486]
{noformat}
java17_pre-commit_tests 
  ✓ j17_build4m 19s
  ✓ j17_cqlsh_dtests_py311   6m 53s
  ✓ j17_cqlsh_dtests_py386m 47s
  ✓ j17_cqlsh_dtests_py38_vnode  6m 55s
  ✓ j17_cqlshlib_cython_tests8m 44s
  ✓ j17_cqlshlib_tests   6m 37s
  ✓ j17_unit_tests  12m 32s
  ✓ j17_unit_tests_repeat6m 40s
  ✓ j17_utests_latest   12m 34s
  ✓ j17_utests_latest_repeat 6m 33s
  ✓ j17_utests_oa   12m 22s
  ✓ j17_utests_oa_repeat 6m 36s
  ✕ j17_cqlsh_dtests_py311_vnode  7m 7s
  ✕ j17_dtests   36m 6s
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j17_dtests_latest   35m 51s
  bootstrap_test.TestBootstrap test_bootstrap_with_reset_bootstrap_state
  offline_tools_test.TestOfflineTools test_sstablelevelreset
  offline_tools_test.TestOfflineTools test_sstableofflinerelevel
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j17_dtests_vnode34m 59s
  gossip_test.TestGossip test_assassinate_valid_node
  ✕ j17_jvm_dtests  34m 36s
  
org.apache.cassandra.distributed.test.NativeTransportEncryptionOptionsTest 
testOptionalMtlsModeDoNotAllowNonSSLConnections
  
org.apache.cassandra.distributed.test.NativeTransportEncryptionOptionsTest 
testEndpointVerificationEnabledIpNotInSAN
  org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest 
coordinatorIsBehindTest TIMEOUTED
  ✕ j17_jvm_dtests_latest_vnode  19m 0s
  junit.framework.TestSuite 
org.apache.cassandra.fuzz.harry.integration.model.InJVMTokenAwareExecutorTest 
TIMEOUTED  
{noformat}

[java17_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4152/workflows/d962e676-1ccd-44d0-b287-cee8162c5c7d]


> enrich system_views.pending_hints with hints sizes
> --
>
> Key: CASSANDRA-19486
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19486
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Consistency/Hints
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
> Fix For: 5.x
>
>  Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> I am happy to close this if somebody shows me how to get total size of all 
> hints or all hints per particular node via JMX.
> I could find StorageMetrics.totalHints but that is how many hints there are, 
> not their sizes. We also have
> org.apache.cassandra.metrics:type=HintedHandOffManager name=
> org.apache.cassandra.metrics:type=HintsService name=
> But that is again showing other metrics not sizes.
> I would add two methods into HintsServiceMBean returning this. Seems to be 
> very easy to do once we do CASSANDRA-19477.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19532 at 4/10/24 12:12 AM:
-

Well ... reading the description on the PR about TCM replaying it with possibly 
changed config ... yeah. We are hitting something similar in CASSANDRA-12937. I 
think this will be problematic more often, everytime we mix TCM transformations 
/ replay with configuration changes ... I am not sure how to solve this.

So we probably need to abandon the introduction of guardrails in 
CreateTriggerStatement.

Neverthless, I still think that the second guardrail might be introduced. It 
would be cool to turn on / off the triggers remotely. 



was (Author: smiklosovic):
Well ... reading the description on the PR about TCM replaying it with possibly 
changed config ... yeah. We are hitting something similar in CASSANDRA-12937. I 
think this will be problematic more often, everytime we mix TCM transformations 
/ replay with configuration changes ... I am not sure how to solve this.

So we probably need to abandon the introduction of guardrails in 
CreateTriggerStatement.

Neverthless, I still think that the second guardrail might be introduced. It 
would be cool to turn on / off the guardrails remotely. 


> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> shortly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19532:
---

Well ... reading the description on the PR about TCM replaying it with possibly 
changed config ... yeah. We are hitting something similar in CASSANDRA-12937. I 
think this will be problematic more often, everytime we mix TCM transformations 
/ replay with configuration changes ... I am not sure how to solve this.

So we probably need to abandon the introduction of guardrails in 
CreateTriggerStatement.

Neverthless, I still think that the second guardrail might be introduced. It 
would be cool to turn on / off the guardrails remotely. 


> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> shortly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Comment Edited] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19532 at 4/9/24 11:28 PM:


It depends how granular we want this to have. This boils down to two 
guardrails. The first one deals with creation, the second one deals with 
incorporating trigger on actual query.

When we take the first one into consideration, the guardrail type would be of 
EnableFlag and it again boils to whether we want to emit a warning or not but 
still enable triggers to be created or we just flat out error out on an attempt 
to create a trigger. If we want to warn but still want to create a trigger we 
need to base this guardrail on the second constructor of EnableFlag taking 
"warned" and "enabled" predicates. Then in cassandra.yaml it would look like 
this:

triggers_create_enabled: true/false
triggers_create_warned: true/false

I think that the first one should be "true" by default. It is questionable what 
you want to have the second one like. If it is "true", then it will by default 
warn upon trigger creation. If this is false, it would not. If the first one is 
"true" then creation of a trigger is possible, if it is false, it will fail the 
query.

The second guardrail, something like "triggers_enabled: true/false", deals with 
including the trigger when a query is executed. I think that should happen as 
the very first thing in TriggerExecutor.execute(mutations) method. The 
guardrail would consist of a simple EnableFlag, the first constructor, taking 
in just one predicate, if it is enabled, trigger would be executed, if it is 
disabled, it wont, but the underlying query itself will not be affected.

I think we need to have two guardrails, because guarding the creation of a 
trigger via CreateTriggerStatement is something else from incorporating the 
execution of it after it is created. For example, an admin might disable 
trigger creation, but she wants to enable existing triggers to be processed. So 
the first guardrail would be disabled, but the second one would be enabled.


was (Author: smiklosovic):
It depends how granular we want this to have. This boils down to two 
guardrails. The first one deals with creation, the second one deals with 
incorporating trigger on actual query.

When we take the first one into consideration, the guardrail type would be of 
EnableFlag and it again boils to whether we want to emit a warning or not but 
still enable triggers to be created or we just flat out error out on an attempt 
to create a trigger. If we want to warn but still want to create a trigger we 
need to base this guardrail on the second constructor of EnableFlag taking 
"warned" and "enabled" predicates. Then in cassandra.yaml it would look like 
this:

triggers_create_enabled: true/false
triggers_create_warned: true/false

I think that the first one should be "true" by default. It is questionable what 
you want to have the second one like. If it is "true", then it will by default 
warn upon trigger creation. If this is false, it would not. If the first one is 
"true" then creation of a trigger is possible, if it is false, it will fail the 
query.

The second guardrail deals with including the trigger when a query is executed. 
I think that should happen as the very first thing in 
TriggerExecutor.execute(mutations) method. The guardrail would consist of a 
simple EnableFlag, the first constructor, taking in just one predicate, if it 
is enabled, trigger would be executed, if it is disabled, it wont, but the 
underlying query itself will not be affected.

I think we need to have two guardrails, because guarding the creation of a 
trigger via CreateTriggerStatement is something else from incorporating the 
execution of it after it is created. For example, an admin might disable 
trigger creation, but she wants to enable existing triggers to be processed. So 
the first guardrail would be disabled, but the second one would be enabled.

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> 

[jira] [Comment Edited] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-19532 at 4/9/24 11:25 PM:


It depends how granular we want this to have. This boils down to two 
guardrails. The first one deals with creation, the second one deals with 
incorporating trigger on actual query.

When we take the first one into consideration, the guardrail type would be of 
EnableFlag and it again boils to whether we want to emit a warning or not but 
still enable triggers to be created or we just flat out error out on an attempt 
to create a trigger. If we want to warn but still want to create a trigger we 
need to base this guardrail on the second constructor of EnableFlag taking 
"warned" and "enabled" predicates. Then in cassandra.yaml it would look like 
this:

triggers_create_enabled: true/false
triggers_create_warned: true/false

I think that the first one should be "true" by default. It is questionable what 
you want to have the second one like. If it is "true", then it will by default 
warn upon trigger creation. If this is false, it would not. If the first one is 
"true" then creation of a trigger is possible, if it is false, it will fail the 
query.

The second guardrail deals with including the trigger when a query is executed. 
I think that should happen as the very first thing in 
TriggerExecutor.execute(mutations) method. The guardrail would consist of a 
simple EnableFlag, the first constructor, taking in just one predicate, if it 
is enabled, trigger would be executed, if it is disabled, it wont, but the 
underlying query itself will not be affected.

I think we need to have two guardrails, because guarding the creation of a 
trigger via CreateTriggerStatement is something else from incorporating the 
execution of it after it is created. For example, an admin might disable 
trigger creation, but she wants to enable existing triggers to be processed. So 
the first guardrail would be disabled, but the second one would be enabled.


was (Author: smiklosovic):
It depends how granular we want this to have. This boils down to two 
guardrails. The first one deals with creation, the second one deals with 
incorporating trigger on actual query.

When we take the first one into consideration, the guardrail type would be of 
EnableFlag and it again boils to whether we want to emit a warning or not but 
still enable triggers to be created or we just flat out error out on an attempt 
to create a trigger. If we want to warn but still want to create a trigger we 
need to base this guardrail on the second constructor of EnableFlag taking 
"warned" and "enabled" predicates. Then in cassandra.yaml it would look like 
this:

triggers_create_enabled: true/false
triggers_warned: true/false

I think that the first one should be "true" by default. It is questionable what 
you want to have the second one like. If it is "true", then it will by default 
warn upon trigger creation. If this is false, it would not. If the first one is 
"true" then creation of a trigger is possible, if it is false, it will fail the 
query.

The second guardrail deals with including the trigger when a query is executed. 
I think that should happen as the very first thing in 
TriggerExecutor.execute(mutations) method. The guardrail would consist of a 
simple EnableFlag, the first constructor, taking in just one predicate, if it 
is enabled, trigger would be executed, if it is disabled, it wont, but the 
underlying query itself will not be affected.

I think we need to have two guardrails, because guarding the creation of a 
trigger via CreateTriggerStatement is something else from incorporating the 
execution of it after it is created. For example, an admin might disable 
trigger creation, but she wants to enable existing triggers to be processed. So 
the first guardrail would be disabled, but the second one would be enabled.

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> shortly.



--
This message was sent by Atlassian Jira

[jira] [Commented] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19532:
---

It depends how granular we want this to have. This boils down to two 
guardrails. The first one deals with creation, the second one deals with 
incorporating trigger on actual query.

When we take the first one into consideration, the guardrail type would be of 
EnableFlag and it again boils to whether we want to emit a warning or not but 
still enable triggers to be created or we just flat out error out on an attempt 
to create a trigger. If we want to warn but still want to create a trigger we 
need to base this guardrail on the second constructor of EnableFlag taking 
"warned" and "enabled" predicates. Then in cassandra.yaml it would look like 
this:

triggers_create_enabled: true/false
triggers_warned: true/false

I think that the first one should be "true" by default. It is questionable what 
you want to have the second one like. If it is "true", then it will by default 
warn upon trigger creation. If this is false, it would not. If the first one is 
"true" then creation of a trigger is possible, if it is false, it will fail the 
query.

The second guardrail deals with including the trigger when a query is executed. 
I think that should happen as the very first thing in 
TriggerExecutor.execute(mutations) method. The guardrail would consist of a 
simple EnableFlag, the first constructor, taking in just one predicate, if it 
is enabled, trigger would be executed, if it is disabled, it wont, but the 
underlying query itself will not be affected.

I think we need to have two guardrails, because guarding the creation of a 
trigger via CreateTriggerStatement is something else from incorporating the 
execution of it after it is created. For example, an admin might disable 
trigger creation, but she wants to enable existing triggers to be processed. So 
the first guardrail would be disabled, but the second one would be enabled.

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> shortly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19532) Allow operators to disable the execution of triggers

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19532:
---

I think that this should be implemented as a new guardrail into existing 
Guardrail framework we already have. This would be also very nicely 
incorporated into CreateTriggerStatement. 

Also, the benefit of using a guardrail is that it might be configured in 
runtime via JMX so an admin can make her mind anytime whether trigger creation 
/ execution is allowed or not. The way it is currently done, we basically need 
to restart a node (and whole cluster) if we want to change it. 

> Allow operators to disable the execution of triggers
> 
>
> Key: CASSANDRA-19532
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19532
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Other
>Reporter: Abe Ratnofsky
>Assignee: Abe Ratnofsky
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Currently, triggers are discouraged but there's no explicit way to disable 
> them. Similar configuration already exists to disable other features, such as 
> "conf.materialized_views_enabled". There should be a means for operators to 
> gracefully disable the creation and execution of triggers.
>  
> I have a patch ready for this, getting a first review now and will push it 
> shortly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19546:
---

That would start to treat every number as a number we want to convert to sizes 
which is not always the case. 

> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables but might be applicable for 
> user tables as well) when a column is of a type which represents number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_human_size(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_human_size(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_human_size(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and represents the number of decimal places 
> (at most) to use. Without the second argument, it will default to 
> FileUtils.df which is "#.##" format.
> {code}
> cqlsh> DESCRIBE myks.mytb ;
> CREATE TABLE myks.mytb (
> id int PRIMARY KEY,
> col1 int,
> col2 smallint,
> col3 bigint,
> col4 varint,
> )
> {code}
> I also propose that this to_human_size function (name of it might be indeed 
> discussed and it is just a suggestion) should be only applicable for int, 
> smallint, bigint and varint types. I am not sure how to apply this to e.g. 
> "float" or similar. As I mentioned, it is meant to convert just number of 
> bytes, which is just some number, to a string representation of that and I do 
> not think that applying that function to anything else but these types makes 
> sense.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19546:
--
Description: 
There are cases (e.g in our system_views tables but might be applicable for 
user tables as well) when a column is of a type which represents number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_human_size(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_human_size(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_human_size(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_human_size function (name of it might be indeed 
discussed and it is just a suggestion) should be only applicable for int, 
smallint, bigint and varint types. I am not sure how to apply this to e.g. 
"float" or similar. As I mentioned, it is meant to convert just number of 
bytes, which is just some number, to a string representation of that and I do 
not think that applying that function to anything else but these types makes 
sense.

  was:
There are cases (e.g in our system_views tables but might be applicable for 
user tables as well) when a column is of a type which represents number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_human_size function (name of it might be indeed 
discussed and it is just a suggestion) should be only applicable for int, 
smallint, bigint and varint types. I am not sure how to apply this to e.g. 
"float" or similar. As I mentioned, it is meant to convert just number of 
bytes, which is just some number, to a string representation of that and I do 
not think that applying that function to anything else but these types makes 
sense.


> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables but might be applicable for 
> user tables as well) when a column is of a type which represents number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_human_size(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_human_size(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_human_size(col4, 1)
> -

[jira] [Commented] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19546:
---

I think the approach I did is just very easy and straightforward to do. I will 
ask on the ML what people think.

Also, I am not sure what setting you have in mind and how that would be applied 
concretely. 

> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables but might be applicable for 
> user tables as well) when a column is of a type which represents number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and represents the number of decimal places 
> (at most) to use. Without the second argument, it will default to 
> FileUtils.df which is "#.##" format.
> {code}
> cqlsh> DESCRIBE myks.mytb ;
> CREATE TABLE myks.mytb (
> id int PRIMARY KEY,
> col1 int,
> col2 smallint,
> col3 bigint,
> col4 varint,
> )
> {code}
> I also propose that this to_human_size function (name of it might be indeed 
> discussed and it is just a suggestion) should be only applicable for int, 
> smallint, bigint and varint types. I am not sure how to apply this to e.g. 
> "float" or similar. As I mentioned, it is meant to convert just number of 
> bytes, which is just some number, to a string representation of that and I do 
> not think that applying that function to anything else but these types makes 
> sense.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Assigned] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic reassigned CASSANDRA-19546:
-

Assignee: Stefan Miklosovic

> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Assignee: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables but might be applicable for 
> user tables as well) when a column is of a type which represents number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and represents the number of decimal places 
> (at most) to use. Without the second argument, it will default to 
> FileUtils.df which is "#.##" format.
> {code}
> cqlsh> DESCRIBE myks.mytb ;
> CREATE TABLE myks.mytb (
> id int PRIMARY KEY,
> col1 int,
> col2 smallint,
> col3 bigint,
> col4 varint,
> )
> {code}
> I also propose that this to_human_size function (name of it might be indeed 
> discussed and it is just a suggestion) should be only applicable for int, 
> smallint, bigint and varint types. I am not sure how to apply this to e.g. 
> "float" or similar. As I mentioned, it is meant to convert just number of 
> bytes, which is just some number, to a string representation of that and I do 
> not think that applying that function to anything else but these types makes 
> sense.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19546:
--
Description: 
There are cases (e.g in our system_views tables but might be applicable for 
user tables as well) when a column is of a type which represents number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_human_size function (name of it might be indeed 
discussed and it is just a suggestion) should be only applicable for int, 
smallint, bigint and varint types. I am not sure how to apply this to e.g. 
"float" or similar. As I mentioned, it is meant to convert just number of 
bytes, which is just some number, to a string representation of that and I do 
not think that applying that function to anything else but these types makes 
sense.

  was:
There are cases (e.g in our system_views tables but might be applicable for 
user tables as well) when a column is of a type which represents number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.


> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables but might be applicable for 
> user tables as well) when a column is of a type which represents number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional 

[jira] [Updated] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19546:
--
Description: 
There are cases (e.g in our system_views tables but might be applicable for 
user tables as well) when a column is of a type which represents number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.

  was:
There are cases (e.g in our system_views tables but might be applicable for 
user tables as well) when a column is of a type which represent number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.


> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables but might be applicable for 
> user tables as well) when a column is of a type which represents number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and 

[jira] [Updated] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19546:
--
Description: 
There are cases (e.g in our system_views tables but might be applicable for 
user tables as well) when a column is of a type which represent number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.

  was:
There are cases (e.g in our system_views tables for might be applicable for 
user tables as well) when a column is of a type which represent number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.


> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables but might be applicable for 
> user tables as well) when a column is of a type which represent number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and 

[jira] [Updated] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19546:
--
Summary: Add to_human_size function  (was: Add to_data function)

> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables for might be applicable for 
> user tables as well) when a column is of a type which represent number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_data(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_data(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_data(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and represents the number of decimal places 
> (at most) to use. Without the second argument, it will default to 
> FileUtils.df which is "#.##" format.
> {code}
> cqlsh> DESCRIBE myks.mytb ;
> CREATE TABLE myks.mytb (
> id int PRIMARY KEY,
> col1 int,
> col2 smallint,
> col3 bigint,
> col4 varint,
> )
> {code}
> I also propose that this to_data function (name of it might be indeed 
> discussed and it is just a suggestion) should be only applicable for int, 
> smallint, bigint and varint types. I am not sure how to apply this to e.g. 
> "float" or similar. As I mentioned, it is meant to convert just number of 
> bytes, which is just some number, to a string representation of that and I do 
> not think that applying that function to anything else but these types makes 
> sense.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19546) Add to_human_size function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19546:
--
Description: 
There are cases (e.g in our system_views tables for might be applicable for 
user tables as well) when a column is of a type which represent number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_human_size(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.

  was:
There are cases (e.g in our system_views tables for might be applicable for 
user tables as well) when a column is of a type which represent number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_data(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_data(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_data(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.


> Add to_human_size function
> --
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables for might be applicable for 
> user tables as well) when a column is of a type which represent number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_human_size(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_human_size(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and represents the number of 

[jira] [Commented] (CASSANDRA-19546) Add to_data function

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19546:
---

Yeah suggest something else if you find it more appropriate ... to_human_size 
or something similar maybe? 

> Add to_data function
> 
>
> Key: CASSANDRA-19546
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/CQL
>Reporter: Stefan Miklosovic
>Priority: Normal
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> There are cases (e.g in our system_views tables for might be applicable for 
> user tables as well) when a column is of a type which represent number of 
> bytes. However, it is quite hard to parse a value for a human to have some 
> estimation what that value is.
> I propose this:
> {code:java}
> cqlsh> select * from myks.mytb ;
>  id | col1 | col2 | col3 | col4 
> +--+--+--+--
>   1 |  100 |  200 |  300 | 32432423 
> (1 rows)
> cqlsh> select to_data(col4) from myks.mytb where id = 1;
>  system.to_data(col4)
> --
> 30.93 MiB
> (1 rows)
> cqlsh> select to_data(col4,0) from myks.mytb where id = 1;
>  system.to_data(col4, 0)
> -
>   31 MiB
> (1 rows)
> cqlsh> select to_data(col4,1) from myks.mytb where id = 1;
>  system.to_data(col4, 1)
> -
> 30.9 MiB
> (1 rows)
> {code}
> The second argument is optional and represents the number of decimal places 
> (at most) to use. Without the second argument, it will default to 
> FileUtils.df which is "#.##" format.
> {code}
> cqlsh> DESCRIBE myks.mytb ;
> CREATE TABLE myks.mytb (
> id int PRIMARY KEY,
> col1 int,
> col2 smallint,
> col3 bigint,
> col4 varint,
> )
> {code}
> I also propose that this to_data function (name of it might be indeed 
> discussed and it is just a suggestion) should be only applicable for int, 
> smallint, bigint and varint types. I am not sure how to apply this to e.g. 
> "float" or similar. As I mentioned, it is meant to convert just number of 
> bytes, which is just some number, to a string representation of that and I do 
> not think that applying that function to anything else but these types makes 
> sense.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Created] (CASSANDRA-19546) Add to_data function

2024-04-09 Thread Stefan Miklosovic (Jira)
Stefan Miklosovic created CASSANDRA-19546:
-

 Summary: Add to_data function
 Key: CASSANDRA-19546
 URL: https://issues.apache.org/jira/browse/CASSANDRA-19546
 Project: Cassandra
  Issue Type: New Feature
  Components: Legacy/CQL
Reporter: Stefan Miklosovic


There are cases (e.g in our system_views tables for might be applicable for 
user tables as well) when a column is of a type which represent number of 
bytes. However, it is quite hard to parse a value for a human to have some 
estimation what that value is.

I propose this:
{code:java}
cqlsh> select * from myks.mytb ;

 id | col1 | col2 | col3 | col4 
+--+--+--+--
  1 |  100 |  200 |  300 | 32432423 

(1 rows)
cqlsh> select to_data(col4) from myks.mytb where id = 1;

 system.to_data(col4)
--
30.93 MiB

(1 rows)
cqlsh> select to_data(col4,0) from myks.mytb where id = 1;

 system.to_data(col4, 0)
-
  31 MiB

(1 rows)
cqlsh> select to_data(col4,1) from myks.mytb where id = 1;

 system.to_data(col4, 1)
-
30.9 MiB

(1 rows)

{code}
The second argument is optional and represents the number of decimal places (at 
most) to use. Without the second argument, it will default to FileUtils.df 
which is "#.##" format.

{code}
cqlsh> DESCRIBE myks.mytb ;

CREATE TABLE myks.mytb (
id int PRIMARY KEY,
col1 int,
col2 smallint,
col3 bigint,
col4 varint,
)
{code}

I also propose that this to_data function (name of it might be indeed discussed 
and it is just a suggestion) should be only applicable for int, smallint, 
bigint and varint types. I am not sure how to apply this to e.g. "float" or 
similar. As I mentioned, it is meant to convert just number of bytes, which is 
just some number, to a string representation of that and I do not think that 
applying that function to anything else but these types makes sense.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19498) Error reading data from credential file

2024-04-09 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19498:
---

[~slavavrn] could you please create patches for 4.1 and 5.0 branches as well? I 
tried to just cherry-pick that from trunk but there are some conflicts I have 
not investigated further. Thank you in advance. 

> Error reading data from credential file
> ---
>
> Key: CASSANDRA-19498
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19498
> Project: Cassandra
>  Issue Type: Bug
>  Components: Documentation, Tool/cqlsh
>Reporter: Slava
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The pylib/cqlshlib/cqlshmain.py code reads data from the credentials file, 
> however, it is immediately ignored.
> https://github.com/apache/cassandra/blob/c9625e0102dab66f41d3ef2338c54d499e73a8c5/pylib/cqlshlib/cqlshmain.py#L2070
> {code:java}
>     if not options.username:
>         credentials = configparser.ConfigParser()
>         if options.credentials is not None:
>             credentials.read(options.credentials)        # use the username 
> from credentials file but fallback to cqlshrc if username is absent from the 
> command line parameters
>         options.username = username_from_cqlshrc    if not options.password:
>         rawcredentials = configparser.RawConfigParser()
>         if options.credentials is not None:
>             rawcredentials.read(options.credentials)        # handling 
> password in the same way as username, priority cli > credentials > cqlshrc
>         options.password = option_with_default(rawcredentials.get, 
> 'plain_text_auth', 'password', password_from_cqlshrc)
>         options.password = password_from_cqlshrc{code}
> These corrections have been made in accordance with 
> https://issues.apache.org/jira/browse/CASSANDRA-16983 and 
> https://issues.apache.org/jira/browse/CASSANDRA-16456.
> The documentation does not indicate that AuthProviders can be used in the 
> cqlshrc and credentials files.
> I propose to return the ability to use the legacy option of specifying the 
> user and password in the credentials file in the [plain_text_auth] section.
> It is also required to describe the rules for using the credentials file in 
> the documentation.
> I can make a corresponding pull request.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19479) Fix type issues and provide tests for type compatibility between 4.1 and 5.0

2024-04-08 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19479:
--
Fix Version/s: 4.0.x
   4.1.x
   5.0.x
   5.x
   (was: 5.0)
   (was: 5.1)
   (was: 4.1.5)
   (was: 4.0.13)

> Fix type issues and provide tests for type compatibility between 4.1 and 5.0
> 
>
> Key: CASSANDRA-19479
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19479
> Project: Cassandra
>  Issue Type: Task
>  Components: Legacy/Core, Test/unit
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>
> This is a part of CASSANDRA-14476 - we should verify whether the type 
> compatibility matrix is upgradable from 4.0 and 4.1 to 5.0, and if not, fix 
> the remaining issues.
> The implemented tests verify the following:
> - assumed compatibility between primitive types
> - equals method symmetricity
> - freezing/unfreezing
> - value compatibility by using a serializer of one type to deserialize a 
> value serialized using a serializer of another type
> - serialization compatibility by serializing a row with a column of one type 
> as a column of another type for simple and complex cells (multicell types)
> - (comparison) compatibility by comparing serialized values of one type using 
> a comparator of another type; for multicell types - build rows and compare 
> cell paths of a complex type using a cell path comparator of another complex 
> type
> - verify whether types that are (value/serialization/comparison) compatible 
> in a previous release are still compatible with this release
> - store the compatibility matrix in a compressed JSON file so that we can 
> copy it to future releases to assert backward compatibility (similar approach 
> to LegacySSTableTest)
> - verify that type serializers are different for non-compatible type pairs 
> which use custom comparisons
> Additionally:
> - the equals method in {{TupleType}} and {{UserType}} was fixed to be 
> symmetric. Previously, comparing two values gave a different outcome when 
> inverted.
> - fixed a condition in comparison method of {{AbstractCompositeType}}
> - ported a fix for composite and dynamic composite types which adds a 
> distinct serializers for them so that the serializers for those types and for 
> {{BytesType}} are considered different; similar thing was done for 
> {{LexicalUUIDType}} to make its serializer different to {{UUIDType}} 
> serializer (see 
> https://the-asf.slack.com/archives/CK23JSY2K/p1712060572432959)
> - fixed a problem with DCT builder - in 5.0+ the {{DynamicCompositeType}} 
> generation has a problem with inverse alias-type mapping which makes it 
> vulnerable to problems when the same type has two different aliases



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19498) Error reading data from credential file

2024-04-08 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19498:
--
Reviewers: Brad Schoening, Stefan Miklosovic, Stefan Miklosovic
   Brad Schoening, Stefan Miklosovic, Stefan Miklosovic  (was: Brad 
Schoening, Stefan Miklosovic)
   Status: Review In Progress  (was: Patch Available)

> Error reading data from credential file
> ---
>
> Key: CASSANDRA-19498
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19498
> Project: Cassandra
>  Issue Type: Bug
>  Components: Documentation, Tool/cqlsh
>Reporter: Slava
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The pylib/cqlshlib/cqlshmain.py code reads data from the credentials file, 
> however, it is immediately ignored.
> https://github.com/apache/cassandra/blob/c9625e0102dab66f41d3ef2338c54d499e73a8c5/pylib/cqlshlib/cqlshmain.py#L2070
> {code:java}
>     if not options.username:
>         credentials = configparser.ConfigParser()
>         if options.credentials is not None:
>             credentials.read(options.credentials)        # use the username 
> from credentials file but fallback to cqlshrc if username is absent from the 
> command line parameters
>         options.username = username_from_cqlshrc    if not options.password:
>         rawcredentials = configparser.RawConfigParser()
>         if options.credentials is not None:
>             rawcredentials.read(options.credentials)        # handling 
> password in the same way as username, priority cli > credentials > cqlshrc
>         options.password = option_with_default(rawcredentials.get, 
> 'plain_text_auth', 'password', password_from_cqlshrc)
>         options.password = password_from_cqlshrc{code}
> These corrections have been made in accordance with 
> https://issues.apache.org/jira/browse/CASSANDRA-16983 and 
> https://issues.apache.org/jira/browse/CASSANDRA-16456.
> The documentation does not indicate that AuthProviders can be used in the 
> cqlshrc and credentials files.
> I propose to return the ability to use the legacy option of specifying the 
> user and password in the credentials file in the [plain_text_auth] section.
> It is also required to describe the rules for using the credentials file in 
> the documentation.
> I can make a corresponding pull request.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Updated] (CASSANDRA-19498) Error reading data from credential file

2024-04-08 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic updated CASSANDRA-19498:
--
Status: Changes Suggested  (was: Review In Progress)

> Error reading data from credential file
> ---
>
> Key: CASSANDRA-19498
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19498
> Project: Cassandra
>  Issue Type: Bug
>  Components: Documentation, Tool/cqlsh
>Reporter: Slava
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The pylib/cqlshlib/cqlshmain.py code reads data from the credentials file, 
> however, it is immediately ignored.
> https://github.com/apache/cassandra/blob/c9625e0102dab66f41d3ef2338c54d499e73a8c5/pylib/cqlshlib/cqlshmain.py#L2070
> {code:java}
>     if not options.username:
>         credentials = configparser.ConfigParser()
>         if options.credentials is not None:
>             credentials.read(options.credentials)        # use the username 
> from credentials file but fallback to cqlshrc if username is absent from the 
> command line parameters
>         options.username = username_from_cqlshrc    if not options.password:
>         rawcredentials = configparser.RawConfigParser()
>         if options.credentials is not None:
>             rawcredentials.read(options.credentials)        # handling 
> password in the same way as username, priority cli > credentials > cqlshrc
>         options.password = option_with_default(rawcredentials.get, 
> 'plain_text_auth', 'password', password_from_cqlshrc)
>         options.password = password_from_cqlshrc{code}
> These corrections have been made in accordance with 
> https://issues.apache.org/jira/browse/CASSANDRA-16983 and 
> https://issues.apache.org/jira/browse/CASSANDRA-16456.
> The documentation does not indicate that AuthProviders can be used in the 
> cqlshrc and credentials files.
> I propose to return the ability to use the legacy option of specifying the 
> user and password in the credentials file in the [plain_text_auth] section.
> It is also required to describe the rules for using the credentials file in 
> the documentation.
> I can make a corresponding pull request.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



[jira] [Commented] (CASSANDRA-19498) Error reading data from credential file

2024-04-08 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-19498:
---

waiting for [~slavavrn] to incorporate [~bschoeni]'s observations.

> Error reading data from credential file
> ---
>
> Key: CASSANDRA-19498
> URL: https://issues.apache.org/jira/browse/CASSANDRA-19498
> Project: Cassandra
>  Issue Type: Bug
>  Components: Documentation, Tool/cqlsh
>Reporter: Slava
>Priority: Normal
> Fix For: 4.1.x, 5.0.x, 5.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The pylib/cqlshlib/cqlshmain.py code reads data from the credentials file, 
> however, it is immediately ignored.
> https://github.com/apache/cassandra/blob/c9625e0102dab66f41d3ef2338c54d499e73a8c5/pylib/cqlshlib/cqlshmain.py#L2070
> {code:java}
>     if not options.username:
>         credentials = configparser.ConfigParser()
>         if options.credentials is not None:
>             credentials.read(options.credentials)        # use the username 
> from credentials file but fallback to cqlshrc if username is absent from the 
> command line parameters
>         options.username = username_from_cqlshrc    if not options.password:
>         rawcredentials = configparser.RawConfigParser()
>         if options.credentials is not None:
>             rawcredentials.read(options.credentials)        # handling 
> password in the same way as username, priority cli > credentials > cqlshrc
>         options.password = option_with_default(rawcredentials.get, 
> 'plain_text_auth', 'password', password_from_cqlshrc)
>         options.password = password_from_cqlshrc{code}
> These corrections have been made in accordance with 
> https://issues.apache.org/jira/browse/CASSANDRA-16983 and 
> https://issues.apache.org/jira/browse/CASSANDRA-16456.
> The documentation does not indicate that AuthProviders can be used in the 
> cqlshrc and credentials files.
> I propose to return the ability to use the legacy option of specifying the 
> user and password in the credentials file in the [plain_text_auth] section.
> It is also required to describe the rules for using the credentials file in 
> the documentation.
> I can make a corresponding pull request.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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



<    1   2   3   4   5   6   7   8   9   10   >