[jira] [Updated] (CASSANDRA-15075) SELECT JSON generates invalid JSON for the duration type

2019-11-28 Thread Marcus Eriksson (Jira)


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

Marcus Eriksson updated CASSANDRA-15075:

  Fix Version/s: (was: 3.11.x)
 (was: 4.x)
 4.0
 3.11.6
  Since Version: 3.11.0
Source Control Link: 
https://github.com/apache/cassandra/commit/5d930cc9db6cdb29c2f7f1dec5a03c5b30ab66a7
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

And committed, thanks!

> SELECT JSON generates invalid JSON for the duration type
> 
>
> Key: CASSANDRA-15075
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15075
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL/Syntax
>Reporter: Pekka Enberg
>Assignee: Pekka Enberg
>Priority: Normal
> Fix For: 3.11.6, 4.0
>
> Attachments: 
> 0001-Fix-SELECT-JSON-formatting-for-the-duration-type.patch
>
>
> Currently, Apache Cassandra generates invalid JSON for the "duration" type.
> cqlsh> CREATE KEYSPACE ks1 WITH REPLICATION = \{ 'class' : 'SimpleStrategy', 
> 'replication_factor' : 1 };
>  cqlsh> CREATE TABLE ks1.data (id int, d duration, PRIMARY KEY (id));
> cqlsh> INSERT INTO ks1.data (id, d) VALUES (1, 6h40m);
>  cqlsh> SELECT JSON d FROM ks1.data WHERE id = 1;
> [json]
>  --
>  \{"d": 6h40m}
> That is, the duration is not quoted and is therefore invalid according to 
> [https://jsonlint.com/,] for example.
>  
> Fix the problem by quoting the formatted duration type properly:
> cqlsh> INSERT INTO ks1.data (id, d) VALUES (1, 6h40m);
>  cqlsh> SELECT JSON d FROM ks1.data WHERE id = 1;
> [json]
>  
>  \{"d": "6h40m"}
> (1 rows)
>  
> The problem is fixed by the following patch:
> [^0001-Fix-SELECT-JSON-formatting-for-the-duration-type.patch]



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

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



[jira] [Commented] (CASSANDRA-15076) Align record header of FQL and audit binary log

2019-11-28 Thread Marcus Eriksson (Jira)


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

Marcus Eriksson commented on CASSANDRA-15076:
-

[~eperott] was preparing to commit this, but noticed 
[this|https://github.com/apache/cassandra/pull/372/files#diff-bdaab1104a93e723ce0b609a6477c9c4L1262-L1266]
 - was that included on purpose?

> Align record header of FQL and audit binary log
> ---
>
> Key: CASSANDRA-15076
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15076
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Tools, Tool/fql
>Reporter: Per Otterström
>Assignee: Per Otterström
>Priority: Normal
> Fix For: 4.0
>
>
> The new full query logger and the audit logger support logging into binary 
> Chronicle logs. Both create records with a small header to indicate what 
> follows, but the two features have adopted different header formats. Let's 
> align the record header format with this ticket.
>  * Both features should use the same header layout. This makes it possible to 
> give more user friendly error messages in the {{fqltool}} and 
> {{auditlogviewr}} commands.
>  * The record header should have a distinct {{type}} to indicate the type of 
> record.
>  * The record header should have a {{version}} so that the record format can 
> evolve.
> Current record header format of the FQL is:
> {noformat}
> version:0(int16)
> type:(text)
> {noformat}
> where {{}} can be either {{batch}} or {{single-query}}.
> Current record header format of the binary audit log is:
> {noformat}
> type:AuditLog(text)
> {noformat}



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

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



[jira] [Comment Edited] (CASSANDRA-15076) Align record header of FQL and audit binary log

2019-11-28 Thread Marcus Eriksson (Jira)


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

Marcus Eriksson edited comment on CASSANDRA-15076 at 11/28/19 8:36 AM:
---

-[~eperott] was preparing to commit this, but noticed 
[this|https://github.com/apache/cassandra/pull/372/files#diff-bdaab1104a93e723ce0b609a6477c9c4L1262-L1266]
 - was that included on purpose?-

edit: yep, leftover from CASSANDRA-14373


was (Author: krummas):
[~eperott] was preparing to commit this, but noticed 
[this|https://github.com/apache/cassandra/pull/372/files#diff-bdaab1104a93e723ce0b609a6477c9c4L1262-L1266]
 - was that included on purpose?

> Align record header of FQL and audit binary log
> ---
>
> Key: CASSANDRA-15076
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15076
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Tools, Tool/fql
>Reporter: Per Otterström
>Assignee: Per Otterström
>Priority: Normal
> Fix For: 4.0
>
>
> The new full query logger and the audit logger support logging into binary 
> Chronicle logs. Both create records with a small header to indicate what 
> follows, but the two features have adopted different header formats. Let's 
> align the record header format with this ticket.
>  * Both features should use the same header layout. This makes it possible to 
> give more user friendly error messages in the {{fqltool}} and 
> {{auditlogviewr}} commands.
>  * The record header should have a distinct {{type}} to indicate the type of 
> record.
>  * The record header should have a {{version}} so that the record format can 
> evolve.
> Current record header format of the FQL is:
> {noformat}
> version:0(int16)
> type:(text)
> {noformat}
> where {{}} can be either {{batch}} or {{single-query}}.
> Current record header format of the binary audit log is:
> {noformat}
> type:AuditLog(text)
> {noformat}



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

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



[jira] [Updated] (CASSANDRA-15076) Align record header of FQL and audit binary log

2019-11-28 Thread Marcus Eriksson (Jira)


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

Marcus Eriksson updated CASSANDRA-15076:

Status: Ready to Commit  (was: Review In Progress)

> Align record header of FQL and audit binary log
> ---
>
> Key: CASSANDRA-15076
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15076
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Tools, Tool/fql
>Reporter: Per Otterström
>Assignee: Per Otterström
>Priority: Normal
> Fix For: 4.0
>
>
> The new full query logger and the audit logger support logging into binary 
> Chronicle logs. Both create records with a small header to indicate what 
> follows, but the two features have adopted different header formats. Let's 
> align the record header format with this ticket.
>  * Both features should use the same header layout. This makes it possible to 
> give more user friendly error messages in the {{fqltool}} and 
> {{auditlogviewr}} commands.
>  * The record header should have a distinct {{type}} to indicate the type of 
> record.
>  * The record header should have a {{version}} so that the record format can 
> evolve.
> Current record header format of the FQL is:
> {noformat}
> version:0(int16)
> type:(text)
> {noformat}
> where {{}} can be either {{batch}} or {{single-query}}.
> Current record header format of the binary audit log is:
> {noformat}
> type:AuditLog(text)
> {noformat}



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

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



[cassandra] branch trunk updated: Align record header of FQL and audit binary log

2019-11-28 Thread marcuse
This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
 new d5e5c45  Align record header of FQL and audit binary log
d5e5c45 is described below

commit d5e5c459f5c4c54c853b5fcfb5c2b3bfeee0d59c
Author: Per Otterström 
AuthorDate: Wed Apr 24 14:41:57 2019 +0200

Align record header of FQL and audit binary log

Patch by Per Otterström; reviewed by Vinay Chella and marcuse for 
CASSANDRA-15076
---
 CHANGES.txt|   1 +
 conf/cassandra.yaml|   5 -
 .../org/apache/cassandra/audit/BinAuditLogger.java |  17 ++-
 .../apache/cassandra/audit/FullQueryLogger.java|  25 ++--
 .../org/apache/cassandra/tools/AuditLogViewer.java |  67 --
 .../org/apache/cassandra/utils/binlog/BinLog.java  |  32 -
 .../apache/cassandra/audit/BinAuditLoggerTest.java |   6 +-
 .../cassandra/audit/FullQueryLoggerTest.java   |  19 ++-
 .../apache/cassandra/tools/AuditLogViewerTest.java | 138 -
 .../apache/cassandra/utils/binlog/BinLogTest.java  |  73 +--
 .../apache/cassandra/fqltool/FQLQueryReader.java   |  38 +-
 .../apache/cassandra/fqltool/commands/Dump.java|  21 +++-
 .../apache/cassandra/fqltool/FQLReplayTest.java|  85 +
 13 files changed, 458 insertions(+), 69 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 43f8cbe..3d1991a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-alpha3
+ * Align record header of FQL and audit binary log (CASSANDRA-15076)
  * Shuffle forwarding replica for messages to non-local DC (CASSANDRA-15318)
  * Optimise native protocol ASCII string encoding (CASSANDRA-15410)
  * Make sure all exceptions are propagated in DebuggableThreadPoolExecutor 
(CASSANDRA-15332)
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index f3e5c75..9a79f24 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -1259,11 +1259,6 @@ back_pressure_strategy:
 # each write which may be lower in order to facilitate availability.
 # ideal_consistency_level: EACH_QUORUM
 
-# Path to write full query log data to when the full query log is enabled
-# The full query log will recrusively delete the contents of this path at
-# times. Don't place links in this directory to other parts of the filesystem.
-#full_query_log_dir: /tmp/cassandrafullquerylog
-
 # Automatically upgrade sstables after upgrade - if there is no ordinary 
compaction to do, the
 # oldest non-upgraded sstable will get upgraded to the latest version
 # automatic_sstable_upgrade: false
diff --git a/src/java/org/apache/cassandra/audit/BinAuditLogger.java 
b/src/java/org/apache/cassandra/audit/BinAuditLogger.java
index 23b9977..83ed3de 100644
--- a/src/java/org/apache/cassandra/audit/BinAuditLogger.java
+++ b/src/java/org/apache/cassandra/audit/BinAuditLogger.java
@@ -30,8 +30,8 @@ import org.apache.cassandra.utils.concurrent.WeightedQueue;
 
 public class BinAuditLogger extends BinLogAuditLogger implements IAuditLogger
 {
-public static final String TYPE = "type";
-public static final String AUDITLOG_TYPE = "AuditLog";
+public static final long CURRENT_VERSION = 0;
+public static final String AUDITLOG_TYPE = "audit";
 public static final String AUDITLOG_MESSAGE = "message";
 
 public BinAuditLogger()
@@ -71,10 +71,19 @@ public class BinAuditLogger extends BinLogAuditLogger 
implements IAuditLogger
 this.message = message;
 }
 
+protected long version()
+{
+return CURRENT_VERSION;
+}
+
+protected String type()
+{
+return AUDITLOG_TYPE;
+}
+
 @Override
-public void writeMarshallable(WireOut wire)
+public void writeMarshallablePayload(WireOut wire)
 {
-wire.write(TYPE).text(AUDITLOG_TYPE);
 wire.write(AUDITLOG_MESSAGE).text(message);
 }
 
diff --git a/src/java/org/apache/cassandra/audit/FullQueryLogger.java 
b/src/java/org/apache/cassandra/audit/FullQueryLogger.java
index 9c1f472..7982940 100644
--- a/src/java/org/apache/cassandra/audit/FullQueryLogger.java
+++ b/src/java/org/apache/cassandra/audit/FullQueryLogger.java
@@ -23,7 +23,6 @@ import java.util.List;
 
 import javax.annotation.Nullable;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Ints;
 
@@ -45,10 +44,7 @@ import org.github.jamm.MemoryLayoutSpecification;
  */
 public class FullQueryLogger extends BinLogAuditLogger implements IAuditLogger
 {
-public static final long CURRENT_VERSION = 0; // encode a dummy version, 
to prevent pain in decoding in the future
-
-public static final String VERSION = "version";
-public static final String TYPE = "type";
+public static 

[jira] [Updated] (CASSANDRA-15076) Align record header of FQL and audit binary log

2019-11-28 Thread Marcus Eriksson (Jira)


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

Marcus Eriksson updated CASSANDRA-15076:

Source Control Link: 
https://github.com/apache/cassandra/commit/d5e5c459f5c4c54c853b5fcfb5c2b3bfeee0d59c
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

committed, ran the tests 
[here|https://circleci.com/workflow-run/ee21bc21-f190-4c8e-ac44-eea36dc1d50b]

> Align record header of FQL and audit binary log
> ---
>
> Key: CASSANDRA-15076
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15076
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Tools, Tool/fql
>Reporter: Per Otterström
>Assignee: Per Otterström
>Priority: Normal
> Fix For: 4.0
>
>
> The new full query logger and the audit logger support logging into binary 
> Chronicle logs. Both create records with a small header to indicate what 
> follows, but the two features have adopted different header formats. Let's 
> align the record header format with this ticket.
>  * Both features should use the same header layout. This makes it possible to 
> give more user friendly error messages in the {{fqltool}} and 
> {{auditlogviewr}} commands.
>  * The record header should have a distinct {{type}} to indicate the type of 
> record.
>  * The record header should have a {{version}} so that the record format can 
> evolve.
> Current record header format of the FQL is:
> {noformat}
> version:0(int16)
> type:(text)
> {noformat}
> where {{}} can be either {{batch}} or {{single-query}}.
> Current record header format of the binary audit log is:
> {noformat}
> type:AuditLog(text)
> {noformat}



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

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



[jira] [Updated] (CASSANDRA-13917) COMPACT STORAGE queries on dense static tables accept hidden column1 and value columns

2019-11-28 Thread Alex Petrov (Jira)


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

Alex Petrov updated CASSANDRA-13917:

 Bug Category: Parent values: Correctness(12982)Level 1 values: API / 
Semantic Implementation(12988)
   Complexity: Challenging  (was: Low Hanging Fruit)
Discovered By: Code Inspection
   Status: Open  (was: Triage Needed)

> COMPACT STORAGE queries on dense static tables accept hidden column1 and 
> value columns
> --
>
> Key: CASSANDRA-13917
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13917
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core
>Reporter: Alex Petrov
>Assignee: Aleksandr Sorokoumov
>Priority: Low
>  Labels: lhf
> Fix For: 3.0.x, 3.11.x
>
> Attachments: 13917-3.0-testall-2.png, 
> 13917-3.0-testall-20.11.2019.png, 13917-3.0.png, 13917-3.11-testall-2.png, 
> 13917-3.11-testall-20.11.2019.png, 13917-3.11.png
>
>
> Test for the issue:
> {code}
> @Test
> public void testCompactStorage() throws Throwable
> {
> createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH 
> COMPACT STORAGE");
> assertInvalid("INSERT INTO %s (a, b, c, column1) VALUES (?, ?, ?, 
> ?)", 1, 1, 1, ByteBufferUtil.bytes('a'));
> // This one fails with Some clustering keys are missing: column1, 
> which is still wrong
> assertInvalid("INSERT INTO %s (a, b, c, value) VALUES (?, ?, ?, ?)", 
> 1, 1, 1, ByteBufferUtil.bytes('a'));   
> assertInvalid("INSERT INTO %s (a, b, c, column1, value) VALUES (?, ?, 
> ?, ?, ?)", 1, 1, 1, ByteBufferUtil.bytes('a'), ByteBufferUtil.bytes('b'));
> assertEmpty(execute("SELECT * FROM %s"));
> }
> {code}
> Gladly, these writes are no-op, even though they succeed.
> {{value}} and {{column1}} should be completely hidden. Fixing this one should 
> be as easy as just adding validations.



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

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



[jira] [Updated] (CASSANDRA-13917) COMPACT STORAGE queries on dense static tables accept hidden column1 and value columns

2019-11-28 Thread Alex Petrov (Jira)


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

Alex Petrov updated CASSANDRA-13917:

Test and Documentation Plan: Unit tests
 Status: Patch Available  (was: Open)

> COMPACT STORAGE queries on dense static tables accept hidden column1 and 
> value columns
> --
>
> Key: CASSANDRA-13917
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13917
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core
>Reporter: Alex Petrov
>Assignee: Aleksandr Sorokoumov
>Priority: Low
>  Labels: lhf
> Fix For: 3.0.x, 3.11.x
>
> Attachments: 13917-3.0-testall-2.png, 
> 13917-3.0-testall-20.11.2019.png, 13917-3.0.png, 13917-3.11-testall-2.png, 
> 13917-3.11-testall-20.11.2019.png, 13917-3.11.png
>
>
> Test for the issue:
> {code}
> @Test
> public void testCompactStorage() throws Throwable
> {
> createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH 
> COMPACT STORAGE");
> assertInvalid("INSERT INTO %s (a, b, c, column1) VALUES (?, ?, ?, 
> ?)", 1, 1, 1, ByteBufferUtil.bytes('a'));
> // This one fails with Some clustering keys are missing: column1, 
> which is still wrong
> assertInvalid("INSERT INTO %s (a, b, c, value) VALUES (?, ?, ?, ?)", 
> 1, 1, 1, ByteBufferUtil.bytes('a'));   
> assertInvalid("INSERT INTO %s (a, b, c, column1, value) VALUES (?, ?, 
> ?, ?, ?)", 1, 1, 1, ByteBufferUtil.bytes('a'), ByteBufferUtil.bytes('b'));
> assertEmpty(execute("SELECT * FROM %s"));
> }
> {code}
> Gladly, these writes are no-op, even though they succeed.
> {{value}} and {{column1}} should be completely hidden. Fixing this one should 
> be as easy as just adding validations.



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

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



[jira] [Updated] (CASSANDRA-13917) COMPACT STORAGE queries on dense static tables accept hidden column1 and value columns

2019-11-28 Thread Alex Petrov (Jira)


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

Alex Petrov updated CASSANDRA-13917:

Status: Triage Needed  (was: Awaiting Feedback)

> COMPACT STORAGE queries on dense static tables accept hidden column1 and 
> value columns
> --
>
> Key: CASSANDRA-13917
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13917
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core
>Reporter: Alex Petrov
>Assignee: Aleksandr Sorokoumov
>Priority: Low
>  Labels: lhf
> Fix For: 3.0.x, 3.11.x
>
> Attachments: 13917-3.0-testall-2.png, 
> 13917-3.0-testall-20.11.2019.png, 13917-3.0.png, 13917-3.11-testall-2.png, 
> 13917-3.11-testall-20.11.2019.png, 13917-3.11.png
>
>
> Test for the issue:
> {code}
> @Test
> public void testCompactStorage() throws Throwable
> {
> createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH 
> COMPACT STORAGE");
> assertInvalid("INSERT INTO %s (a, b, c, column1) VALUES (?, ?, ?, 
> ?)", 1, 1, 1, ByteBufferUtil.bytes('a'));
> // This one fails with Some clustering keys are missing: column1, 
> which is still wrong
> assertInvalid("INSERT INTO %s (a, b, c, value) VALUES (?, ?, ?, ?)", 
> 1, 1, 1, ByteBufferUtil.bytes('a'));   
> assertInvalid("INSERT INTO %s (a, b, c, column1, value) VALUES (?, ?, 
> ?, ?, ?)", 1, 1, 1, ByteBufferUtil.bytes('a'), ByteBufferUtil.bytes('b'));
> assertEmpty(execute("SELECT * FROM %s"));
> }
> {code}
> Gladly, these writes are no-op, even though they succeed.
> {{value}} and {{column1}} should be completely hidden. Fixing this one should 
> be as easy as just adding validations.



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

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



[jira] [Commented] (CASSANDRA-13917) COMPACT STORAGE queries on dense static tables accept hidden column1 and value columns

2019-11-28 Thread Alex Petrov (Jira)


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

Alex Petrov commented on CASSANDRA-13917:
-

Thank you for the patch [~Ge]. I think the patch is overall good, but I was 
a bit skeptical about {{hiddenColumns}} set and its creation depending on 
{{isStaticCompactTable}}, which has lead me to consider other cases, such as:

{code}
createTable("CREATE TABLE %s (a int, b int, PRIMARY KEY(a, b)) WITH 
COMPACT STORAGE");
execute("INSERT INTO %s (a, b, value) VALUES (?, ?, ?)", 1, 1, null); 
// this should not work
{code}

I did check some thrift-created tables, but in the most other things work the 
same way with and without compact storage. You can get some more information 
and (hopefully) an exhaustive set of compact storage table examples in 
{{DropCompactStorageThriftTest}}. 

In short, we should make sure that non-static compact tables also work as 
expected while not breaking dense tables that actually have the value column 
specified. This means that we should probably take a slightly different 
approach for validation and check for hidden columns depending on the table 
configuration.

Supercolumn families seem to work fine; but I also think we probably can skip 
adding tests for those.

> COMPACT STORAGE queries on dense static tables accept hidden column1 and 
> value columns
> --
>
> Key: CASSANDRA-13917
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13917
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core
>Reporter: Alex Petrov
>Assignee: Aleksandr Sorokoumov
>Priority: Low
>  Labels: lhf
> Fix For: 3.0.x, 3.11.x
>
> Attachments: 13917-3.0-testall-2.png, 
> 13917-3.0-testall-20.11.2019.png, 13917-3.0.png, 13917-3.11-testall-2.png, 
> 13917-3.11-testall-20.11.2019.png, 13917-3.11.png
>
>
> Test for the issue:
> {code}
> @Test
> public void testCompactStorage() throws Throwable
> {
> createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH 
> COMPACT STORAGE");
> assertInvalid("INSERT INTO %s (a, b, c, column1) VALUES (?, ?, ?, 
> ?)", 1, 1, 1, ByteBufferUtil.bytes('a'));
> // This one fails with Some clustering keys are missing: column1, 
> which is still wrong
> assertInvalid("INSERT INTO %s (a, b, c, value) VALUES (?, ?, ?, ?)", 
> 1, 1, 1, ByteBufferUtil.bytes('a'));   
> assertInvalid("INSERT INTO %s (a, b, c, column1, value) VALUES (?, ?, 
> ?, ?, ?)", 1, 1, 1, ByteBufferUtil.bytes('a'), ByteBufferUtil.bytes('b'));
> assertEmpty(execute("SELECT * FROM %s"));
> }
> {code}
> Gladly, these writes are no-op, even though they succeed.
> {{value}} and {{column1}} should be completely hidden. Fixing this one should 
> be as easy as just adding validations.



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

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



[jira] [Updated] (CASSANDRA-13917) COMPACT STORAGE queries on dense static tables accept hidden column1 and value columns

2019-11-28 Thread Alex Petrov (Jira)


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

Alex Petrov updated CASSANDRA-13917:

Status: Changes Suggested  (was: Review In Progress)

> COMPACT STORAGE queries on dense static tables accept hidden column1 and 
> value columns
> --
>
> Key: CASSANDRA-13917
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13917
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core
>Reporter: Alex Petrov
>Assignee: Aleksandr Sorokoumov
>Priority: Low
>  Labels: lhf
> Fix For: 3.0.x, 3.11.x
>
> Attachments: 13917-3.0-testall-2.png, 
> 13917-3.0-testall-20.11.2019.png, 13917-3.0.png, 13917-3.11-testall-2.png, 
> 13917-3.11-testall-20.11.2019.png, 13917-3.11.png
>
>
> Test for the issue:
> {code}
> @Test
> public void testCompactStorage() throws Throwable
> {
> createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH 
> COMPACT STORAGE");
> assertInvalid("INSERT INTO %s (a, b, c, column1) VALUES (?, ?, ?, 
> ?)", 1, 1, 1, ByteBufferUtil.bytes('a'));
> // This one fails with Some clustering keys are missing: column1, 
> which is still wrong
> assertInvalid("INSERT INTO %s (a, b, c, value) VALUES (?, ?, ?, ?)", 
> 1, 1, 1, ByteBufferUtil.bytes('a'));   
> assertInvalid("INSERT INTO %s (a, b, c, column1, value) VALUES (?, ?, 
> ?, ?, ?)", 1, 1, 1, ByteBufferUtil.bytes('a'), ByteBufferUtil.bytes('b'));
> assertEmpty(execute("SELECT * FROM %s"));
> }
> {code}
> Gladly, these writes are no-op, even though they succeed.
> {{value}} and {{column1}} should be completely hidden. Fixing this one should 
> be as easy as just adding validations.



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

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



[jira] [Updated] (CASSANDRA-13917) COMPACT STORAGE queries on dense static tables accept hidden column1 and value columns

2019-11-28 Thread Alex Petrov (Jira)


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

Alex Petrov updated CASSANDRA-13917:

Reviewers: Alex Petrov, Alex Petrov  (was: Alex Petrov)
   Alex Petrov, Alex Petrov  (was: Alex Petrov)
   Status: Review In Progress  (was: Patch Available)

> COMPACT STORAGE queries on dense static tables accept hidden column1 and 
> value columns
> --
>
> Key: CASSANDRA-13917
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13917
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core
>Reporter: Alex Petrov
>Assignee: Aleksandr Sorokoumov
>Priority: Low
>  Labels: lhf
> Fix For: 3.0.x, 3.11.x
>
> Attachments: 13917-3.0-testall-2.png, 
> 13917-3.0-testall-20.11.2019.png, 13917-3.0.png, 13917-3.11-testall-2.png, 
> 13917-3.11-testall-20.11.2019.png, 13917-3.11.png
>
>
> Test for the issue:
> {code}
> @Test
> public void testCompactStorage() throws Throwable
> {
> createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH 
> COMPACT STORAGE");
> assertInvalid("INSERT INTO %s (a, b, c, column1) VALUES (?, ?, ?, 
> ?)", 1, 1, 1, ByteBufferUtil.bytes('a'));
> // This one fails with Some clustering keys are missing: column1, 
> which is still wrong
> assertInvalid("INSERT INTO %s (a, b, c, value) VALUES (?, ?, ?, ?)", 
> 1, 1, 1, ByteBufferUtil.bytes('a'));   
> assertInvalid("INSERT INTO %s (a, b, c, column1, value) VALUES (?, ?, 
> ?, ?, ?)", 1, 1, 1, ByteBufferUtil.bytes('a'), ByteBufferUtil.bytes('b'));
> assertEmpty(execute("SELECT * FROM %s"));
> }
> {code}
> Gladly, these writes are no-op, even though they succeed.
> {{value}} and {{column1}} should be completely hidden. Fixing this one should 
> be as easy as just adding validations.



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

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



[jira] [Updated] (CASSANDRA-15350) Add CAS “uncertainty” and “contention" messages that are currently propagated as a WriteTimeoutException.

2019-11-28 Thread Alex Petrov (Jira)


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

Alex Petrov updated CASSANDRA-15350:

Change Category: Operability
 Complexity: Normal
 Status: Open  (was: Triage Needed)

> Add CAS “uncertainty” and “contention" messages that are currently propagated 
> as a WriteTimeoutException.
> -
>
> Key: CASSANDRA-15350
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15350
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Feature/Lightweight Transactions
>Reporter: Alex Petrov
>Assignee: Yifan Cai
>Priority: Normal
>  Labels: protocolv5, pull-request-available
> Attachments: Utf8StringEncodeBench.java
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Right now, CAS uncertainty introduced in 
> https://issues.apache.org/jira/browse/CASSANDRA-6013 is propagating as 
> WriteTimeout. One of this conditions it manifests is when there’s at least 
> one acceptor that has accepted the value, which means that this value _may_ 
> still get accepted during the later round, despite the proposer failure. 
> Similar problem happens with CAS contention, which is also indistinguishable 
> from the “regular” timeout, even though it is visible in metrics correctly.



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

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



[jira] [Updated] (CASSANDRA-15295) Running into deadlock when do CommitLog initialization

2019-11-28 Thread Dinesh Joshi (Jira)


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

Dinesh Joshi updated CASSANDRA-15295:
-
Status: Review In Progress  (was: Changes Suggested)

> Running into deadlock when do CommitLog initialization
> --
>
> Key: CASSANDRA-15295
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15295
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Commit Log
>Reporter: Zephyr Guo
>Assignee: Zephyr Guo
>Priority: Normal
> Attachments: image.png, jstack.log, pstack.log, screenshot-1.png, 
> screenshot-2.png, screenshot-3.png
>
>
> Recently, I found a cassandra(3.11.4) node stuck in STARTING status for a 
> long time.
>  I used jstack to saw what happened. The main thread stuck in 
> *AbstractCommitLogSegmentManager.awaitAvailableSegment*
>  !screenshot-1.png! 
> The strange thing is COMMIT-LOG-ALLOCATOR thread state was runnable but it 
> was not actually running.  
>  !screenshot-2.png! 
> And then I used pstack to troubleshoot. I found COMMIT-LOG-ALLOCATOR block on 
> java class initialization.
>   !screenshot-3.png! 
> This is a deadlock obviously. CommitLog waits for a CommitLogSegment when 
> initializing. In this moment, the CommitLog class is not initialized and the 
> main thread holds the class lock. After that, COMMIT-LOG-ALLOCATOR creates a 
> CommitLogSegment with exception and call *CommitLog.handleCommitError*(static 
> method).  COMMIT-LOG-ALLOCATOR will block on this line because CommitLog 
> class is still initializing.
>  
>  



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

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



[jira] [Commented] (CASSANDRA-14365) Commit log replay failure for static columns with collections in clustering keys

2019-11-28 Thread Michael Semb Wever (Jira)


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

Michael Semb Wever commented on CASSANDRA-14365:


With new tests… (test against trunk also needed a rewrite bc 
{{`TableMetadata.Builder`}}

||branch||circleci||asf jenkins tests||asf jenkins dtests||
|[cassandra-2.2_14365|https://github.com/apache/cassandra/compare/cassandra-2.2...thelastpickle:mck/cassandra-2.2_14365]|[circleci|https://circleci.com/workflow-run/d500cc5f-1d87-4beb-815e-9931f8e84d95]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/29//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/29/]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/703//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/703]|
|[cassandra-3.0_14365|https://github.com/apache/cassandra/compare/cassandra-3.0...thelastpickle:mck/cassandra-3.0_14365]|[circleci|https://circleci.com/workflow-run/747730de-573a-4e80-98f0-4defa14db909]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/33//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/33/]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/706//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/706]|
|[cassandra-3.11_14365|https://github.com/apache/cassandra/compare/cassandra-3.11...thelastpickle:mck/cassandra-3.11_14365]|[circleci|https://circleci.com/workflow-run/86ca8a61-5cc2-40db-84a4-1210cf44f285]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/34//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/34/]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/707//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/707]|
|[trunk_14365|https://github.com/apache/cassandra/compare/trunk...thelastpickle:mck/trunk_14365]|[circleci|https://circleci.com/workflow-run/a034a6b1-a7d7-43cd-b1ab-14769799b30e]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/35//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-pipeline/35/]|[!https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/707//badge/icon!|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/707]|

> Commit log replay failure for static columns with collections in clustering 
> keys
> 
>
> Key: CASSANDRA-14365
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14365
> Project: Cassandra
>  Issue Type: Bug
>  Components: Legacy/Core
>Reporter: Vincent White
>Assignee: Vincent White
>Priority: Normal
>
> In the old storage engine, static cells with a collection as part of the 
> clustering key fail to validate because a 0 byte collection (like in the cell 
> name of a static cell) isn't valid.
> To reproduce:
> 1.
> {code:java}
> CREATE TABLE test.x (
> id int,
> id2 frozen>,
> st int static,
> PRIMARY KEY (id, id2)
> );
> INSERT INTO test.x (id, st) VALUES (1, 2);
> {code}
> 2.
>  Kill the cassandra process
> 3.
>  Restart cassandra to replay the commitlog
> Outcome:
> {noformat}
> ERROR [main] 2018-04-05 04:58:23,741 JVMStabilityInspector.java:99 - Exiting 
> due to error while processing commit log during initialization.
> org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: 
> Unexpected error deserializing mutation; saved to 
> /tmp/mutation3825739904516830950dat.  This may be caused by replaying a 
> mutation against a table with the same name but incompatible schema.  
> Exception follows: org.apache.cassandra.serializers.MarshalException: Not 
> enough bytes to read a set
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:638)
>  [main/:na]
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:565)
>  [main/:na]
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:517)
>  [main/:na]
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:397)
>  [main/:na]
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:143)
>  [main/:na]
> at 
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:181) 
> [main/:na]
> at 
> org.apache.cassandr