[jira] [Commented] (CASSANDRA-11979) cqlsh copyutil should get host metadata by connected address

2016-07-03 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-11979:
--

Driver version 3.5 is now available and will be embedded into cqlsh by 
CASSANDRA-11850, once that ticket is committed this task can be done.

> cqlsh copyutil should get host metadata by connected address
> 
>
> Key: CASSANDRA-11979
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11979
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Adam Holmberg
>Assignee: Stefania
>Priority: Minor
> Fix For: 2.2.x, 3.x
>
>
> pylib.copyutil presently accesses cluster metadata using {{shell.hostname}} 
> which could be an unresolved hostname.
> https://github.com/apache/cassandra/blob/58d3b9a90461806d44dd85bf4aa928e575d5fb6c/pylib/cqlshlib/copyutil.py#L207
> Cluster metadata normally refers to hosts in terms of numeric host address, 
> not hostname. This works in the current integration because the driver allows 
> hosts with unresolved names into metadata during the initial control 
> connection. In a future version of the driver, that anomaly is removed, and 
> no duplicate hosts-by-name are present in the metadata.
> We will need to update copyutil to refer to hosts by address when accessing 
> metadata. This can be accomplished by one of two methods presently:
> # shell.conn.control_connection.host (gives the current connected host 
> address)
> # scan metadata.all_hosts() for the one that {{is_up}} and use 
> host.address/host.datacenter



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11996) SSTableSet.CANONICAL can miss sstables

2016-07-03 Thread Stefania (JIRA)

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

Stefania updated CASSANDRA-11996:
-
Status: Ready to Commit  (was: Patch Available)

> SSTableSet.CANONICAL can miss sstables
> --
>
> Key: CASSANDRA-11996
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11996
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Critical
> Fix For: 3.0.x, 3.x
>
>
> There is a race where we might miss sstables in SSTableSet.CANONICAL when we 
> finish up a compaction.
> Reproducing unit test pushed 
> [here|https://github.com/krummas/cassandra/commit/1292aaa61b89730cff0c022ed1262f45afd493e5]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11996) SSTableSet.CANONICAL can miss sstables

2016-07-03 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-11996:
--

utests look good now, all failures happening on unpatched branches as well.

Ticket can be committed as soon as CASSANDRA-11944 is ready.

> SSTableSet.CANONICAL can miss sstables
> --
>
> Key: CASSANDRA-11996
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11996
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Critical
> Fix For: 3.0.x, 3.x
>
>
> There is a race where we might miss sstables in SSTableSet.CANONICAL when we 
> finish up a compaction.
> Reproducing unit test pushed 
> [here|https://github.com/krummas/cassandra/commit/1292aaa61b89730cff0c022ed1262f45afd493e5]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11850) cannot use cql since upgrading python to 2.7.11+

2016-07-03 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-11850:
--

I've reproduced the problem with python 2.7.12 and cassandra 2.1+. I've also 
verified that the latest python driver, 3.5.0, fixes the problem for 2.2+ 
whilst for 2.1 I've created a pull request 
[here|https://github.com/datastax/python-driver/pull/606] to add the fix to the 
driver 2.7.2 branch.

I've launched the cqlsh tests with the updated driver:

||2.1||2.2||3.0||3.9||trunk||
|[patch|https://github.com/stef1927/cassandra/commits/11850-cqlsh-2.1]|[patch|https://github.com/stef1927/cassandra/commits/11850-cqlsh-2.2]|[patch|https://github.com/stef1927/cassandra/commits/11850-cqlsh-3.0]|[patch|https://github.com/stef1927/cassandra/commits/11850-cqlsh-3.9]|[patch|https://github.com/stef1927/cassandra/commits/11850-cqlsh]|
|[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11850-cqlsh-2.1-cqlsh-tests/]|[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11850-cqlsh-2.2-cqlsh-tests/]|[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11850-cqlsh-3.0-cqlsh-tests/]|[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11850-cqlsh-3.9-cqlsh-tests/]|[dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-11850-cqlsh-cqlsh-tests/]|

I'm hoping that the Unicode problems I previously observed with the driver 
version 3.4 are now resolved. If so we can commit, otherwise I'll bump the 
urgency of this ticket given that that problem now occurs with a released 
python version.

> cannot use cql since upgrading python to 2.7.11+
> 
>
> Key: CASSANDRA-11850
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11850
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
> Environment: Development
>Reporter: Andrew Madison
>Assignee: Stefania
>  Labels: cqlsh
> Fix For: 2.2.x, 3.0.x, 3.x
>
>
> OS: Debian GNU/Linux stretch/sid 
> Kernel: 4.5.0-2-amd64 #1 SMP Debian 4.5.4-1 (2016-05-16) x86_64 GNU/Linux
> Python version: 2.7.11+ (default, May  9 2016, 15:54:33)
> [GCC 5.3.1 20160429]
> cqlsh --version: cqlsh 5.0.1
> cassandra -v: 3.5 (also occurs with 3.0.6)
> Issue:
> when running cqlsh, it returns the following error:
> cqlsh -u dbarpt_usr01
> Password: *
> Connection error: ('Unable to connect to any servers', {'odbasandbox1': 
> TypeError('ref() does not take keyword arguments',)})
> I cleared PYTHONPATH:
> python -c "import json; print dir(json); print json.__version__"
> ['JSONDecoder', 'JSONEncoder', '__all__', '__author__', '__builtins__', 
> '__doc__', '__file__', '__name__', '__package__', '__path__', '__version__', 
> '_default_decoder', '_default_encoder', 'decoder', 'dump', 'dumps', 
> 'encoder', 'load', 'loads', 'scanner']
> 2.0.9
> Java based clients can connect to Cassandra with no issue. Just CQLSH and 
> Python clients cannot.
> nodetool status also works.
> Thank you for your help.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator

2016-07-03 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-9318:
-

bq. For that specific test I've got no client timeouts at all, as I wrote at 
ONE.

Sorry I should have been clearer, I meant what were the 
{{write_request_timeout_in_ms}} and {{back_pressure_timeout_override}} yaml 
settings?

bq. Agreed with all your points. I'll see what I can do, but any help/pointers 
will be very appreciated.

We can do the following:

bq. verify we can reduce the number of dropped mutations in a larger (5-10 
nodes) cluster with multiple clients writing simultaneously

I will ask for help to the TEs, more details to follow.

bq. some cstar perf tests to ensure ops per second are not degraded, both read 
and writes

We can launch a comparison test [here|http://cstar.datastax.com], 30M rows 
should be enough. I can launch it for you if you don't have an account.

bq. the dtests should be run with and without backpressure enabled

This can be done by temporarily changing cassandra.yaml on your branch and then 
launching the dtests.

bq. we should do a bulk load test, for example for cqlsh COPY FROM

I can take care of this. I don't expect problems because COPY FROM should 
contact the replicas directly, it's just a box I want to tick. Importing 5 to 
10M rows with 3 nodes should be sufficient.

bq. Please send me a PR and I'll incorporate those in my branch

I couldn't create a PR, for some reason sbtourist/cassandra wasn't in the base 
fork list. I've attached a patch to this ticket, 
[^9318-3.0-nits-trailing-spaces.patch].

bq. I find the current layout effective and simple enough, but I'll not object 
if you want to push those under a common "container" option.

The encryption options are what I was aiming at, but it's true that for 
everything else we have a flat layout, so let's leave it as it is.

bq. I don't like much that name either, as it doesn't convey very well the 
(double) meaning; making the back-pressure window the same as the write timeout 
is not strictly necessary, but it makes the algorithm behave better in terms of 
reducing dropped mutations as it gives replica more time to process its backlog 
after the rate is reduced. Let me think about that a bit more, but I'd like to 
avoid requiring the user to increase the write timeout manually, as again, it 
reduces the effectiveness of the algorithm.

I'll let you think about it. Maybe a boolean property that is true by default 
and that clearly indicates that the timeout is overridden, although this 
complicates things somewhat.

bq. Sure I can switch to that on trunk, if you think it's worth 
performance-wise (I can write a JMH test if there isn't one already).

The precision is only 10 milliseconds, if this is acceptable it would be 
interesting to see what the difference in performance is.

bq. It is not used in any unit tests code, but it is used in my manual byteman 
tests, and unfortunately I need it on the C* classpath; is that a problem to 
keep it?

Sorry I missed the byteman imports and helper. Let's just move it to the test 
source folder and add a comment. 

--

The rest of the CR points are fine. 

One thing we did not confirm is whether you are happy committing this only to 
trunk or whether you need this in 3.0. Strictly speaking 3.0 accepts only bug 
fixes, not new features. However, this is an optional feature that solves a 
problem (dropped mutations) and that is disabled by default, so we have a case 
for an exception.

> Bound the number of in-flight requests at the coordinator
> -
>
> Key: CASSANDRA-9318
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9318
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local Write-Read Paths, Streaming and Messaging
>Reporter: Ariel Weisberg
>Assignee: Sergio Bossa
> Attachments: 9318-3.0-nits-trailing-spaces.patch, backpressure.png, 
> limit.btm, no_backpressure.png
>
>
> It's possible to somewhat bound the amount of load accepted into the cluster 
> by bounding the number of in-flight requests and request bytes.
> An implementation might do something like track the number of outstanding 
> bytes and requests and if it reaches a high watermark disable read on client 
> connections until it goes back below some low watermark.
> Need to make sure that disabling read on the client connection won't 
> introduce other issues.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator

2016-07-03 Thread Stefania (JIRA)

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

Stefania updated CASSANDRA-9318:

Attachment: 9318-3.0-nits-trailing-spaces.patch

> Bound the number of in-flight requests at the coordinator
> -
>
> Key: CASSANDRA-9318
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9318
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local Write-Read Paths, Streaming and Messaging
>Reporter: Ariel Weisberg
>Assignee: Sergio Bossa
> Attachments: 9318-3.0-nits-trailing-spaces.patch, backpressure.png, 
> limit.btm, no_backpressure.png
>
>
> It's possible to somewhat bound the amount of load accepted into the cluster 
> by bounding the number of in-flight requests and request bytes.
> An implementation might do something like track the number of outstanding 
> bytes and requests and if it reaches a high watermark disable read on client 
> connections until it goes back below some low watermark.
> Need to make sure that disabling read on the client connection won't 
> introduce other issues.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11996) SSTableSet.CANONICAL can miss sstables

2016-07-03 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-11996:
--

Patch LGTM but the utests failed to launch on all branches so I've restarted 
them. dtest failures are all accounted for (either known failures, or 
connection timeouts). 

> SSTableSet.CANONICAL can miss sstables
> --
>
> Key: CASSANDRA-11996
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11996
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Critical
> Fix For: 3.0.x, 3.x
>
>
> There is a race where we might miss sstables in SSTableSet.CANONICAL when we 
> finish up a compaction.
> Reproducing unit test pushed 
> [here|https://github.com/krummas/cassandra/commit/1292aaa61b89730cff0c022ed1262f45afd493e5]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11990) Address rows rather than partitions in SASI

2016-07-03 Thread Alex Petrov (JIRA)

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

Alex Petrov commented on CASSANDRA-11990:
-

[~doanduyhai] I know this has a big impact on the storage engine. However, 
implementing {{Unfiltered  SSTableReader.rowAt(long rowOffset)}} is 
non-trivial, as it'll also involve extending the token for clustering order. 
I've opted out for a simpler approach for now, which is only reading out the 
clustering keys and using them for sorting for sakes of proof-of-concept. This 
change was very small and non-intrusive and leaves us enough room to improve it 
in future.

> Address rows rather than partitions in SASI
> ---
>
> Key: CASSANDRA-11990
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11990
> Project: Cassandra
>  Issue Type: Improvement
>  Components: CQL
>Reporter: Alex Petrov
>Assignee: Alex Petrov
>
> Currently, the lookup in SASI index would return the key position of the 
> partition. After the partition lookup, the rows are iterated and the 
> operators are applied in order to filter out ones that do not match.
> bq. TokenTree which accepts variable size keys (such would enable different 
> partitioners, collections support, primary key indexing etc.), 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11990) Address rows rather than partitions in SASI

2016-07-03 Thread DOAN DuyHai (JIRA)

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

DOAN DuyHai commented on CASSANDRA-11990:
-

[~ifesdjeen]  There is also a big change in {{SSTableReader}} that is required 
to implement this JIRA. Currently this class only exposes the method 
{{DecoratedKey  SSTableReader.keyAt(long indexPosition)}}. We'll need to 
introduce a new method {{Unfiltered SSTableReader.rowAt(long rowOffset)}}.

 Modifying the {{SSTableReader}} has a big impact on the overall storage engine 

> Address rows rather than partitions in SASI
> ---
>
> Key: CASSANDRA-11990
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11990
> Project: Cassandra
>  Issue Type: Improvement
>  Components: CQL
>Reporter: Alex Petrov
>Assignee: Alex Petrov
>
> Currently, the lookup in SASI index would return the key position of the 
> partition. After the partition lookup, the rows are iterated and the 
> operators are applied in order to filter out ones that do not match.
> bq. TokenTree which accepts variable size keys (such would enable different 
> partitioners, collections support, primary key indexing etc.), 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12128) Unexpected amount of returned results after querying SASI index

2016-07-03 Thread Alex Petrov (JIRA)

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

Alex Petrov commented on CASSANDRA-12128:
-

My bad, I have misread the condition, it was {{%a}}, which means "ends with". 
Which raises the question if it's possible to do the term query without 
post-filtering, going to investigate this.

> Unexpected amount of returned results after querying SASI index
> ---
>
> Key: CASSANDRA-12128
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12128
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Alex Petrov
>
> I'm working on [CASSANDRA-11990] and one of the unit tests started failing 
> after the changes I've made. However, it seems that the results returned on 
> {{trunk}} are incorrect. 
> One can reproduce it with the following code:
> {code}
> @Test
> public void testCrossSSTableQueries2() throws Exception
> {
> Map> part1 = new HashMap Pair>()
> {{
> put("key0", Pair.create("Maxie", 43));
> put("key1", Pair.create("Chelsie", 33));
> put("key2", Pair.create("Josephine", 43));
> put("key3", Pair.create("Shanna", 27));
> put("key4", Pair.create("Amiya", 36));
> }};
> loadData(part1, true); // first sstable
> Map> part2 = new HashMap Pair>()
> {{
> put("key5", Pair.create("Americo", 20));
> put("key6", Pair.create("Fiona", 39));
> put("key7", Pair.create("Francis", 41));
> put("key8", Pair.create("Charley", 21));
> put("key9", Pair.create("Amely", 40));
> }};
> loadData(part2, true);
> Map> part3 = new HashMap Pair>()
> {{
> put("key10", Pair.create("Eddie", 42));
> put("key11", Pair.create("Oswaldo", 35));
> put("key12", Pair.create("Susana", 35));
> put("key13", Pair.create("Alivia", 42));
> put("key14", Pair.create("Demario", 28));
> }};
> ColumnFamilyStore store = loadData(part3, true);
> final ByteBuffer firstName = 
> UTF8Type.instance.decompose("first_name");
> Set rows;
> rows = getIndexed(store, 10,
>   buildExpression(firstName, Operator.LIKE_SUFFIX, 
> UTF8Type.instance.decompose("a")));
> Map> concat = new HashMap<>();
> concat.putAll(part1);
> concat.putAll(part2);
> concat.putAll(part3);
> Set keys = new HashSet<>();
> concat.forEach((k, v) -> {
> int idx = v.left.indexOf('a');
> if (idx > 0)
> {
> keys.add(k);
> }
> });
> System.out.println("keys = " + keys);
> String[] arr = new String[keys.size()];
> keys.toArray(arr);
> Assert.assertTrue(rows.toString(), Arrays.equals(arr, 
> rows.toArray(new String[rows.size()])));
> }
> {code}
> Expected results are (emphasis mine)
> {code}
> [key14, key13, key0, key12, key6, key3, key4, key7, key8, key11]
> "key14"  "Dem_a_rio" 
> "key13"  "Alivi_a_" 
> "key0"   "M_a_xie" 
> "key12"  "Sus_a_na" 
> "key6"   "Fion_a_" 
> "key3"   "Sh_a_nna" 
> "key4"   "Amiy_a_" 
> "key7"   "Fr_a_ncis"
> "key8"   "Ch_a_rley" 
> "key11"  "Osw_a_ldo" 
> {code}
> although returned are
> {code}
> [key12, key13, key3, key4, key6]
> {code}
> cc [~xedin] [~doanduyhai] [~beobal] 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (CASSANDRA-12128) Unexpected amount of returned results after querying SASI index

2016-07-03 Thread Alex Petrov (JIRA)

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

Alex Petrov resolved CASSANDRA-12128.
-
Resolution: Invalid

> Unexpected amount of returned results after querying SASI index
> ---
>
> Key: CASSANDRA-12128
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12128
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Alex Petrov
>
> I'm working on [CASSANDRA-11990] and one of the unit tests started failing 
> after the changes I've made. However, it seems that the results returned on 
> {{trunk}} are incorrect. 
> One can reproduce it with the following code:
> {code}
> @Test
> public void testCrossSSTableQueries2() throws Exception
> {
> Map> part1 = new HashMap Pair>()
> {{
> put("key0", Pair.create("Maxie", 43));
> put("key1", Pair.create("Chelsie", 33));
> put("key2", Pair.create("Josephine", 43));
> put("key3", Pair.create("Shanna", 27));
> put("key4", Pair.create("Amiya", 36));
> }};
> loadData(part1, true); // first sstable
> Map> part2 = new HashMap Pair>()
> {{
> put("key5", Pair.create("Americo", 20));
> put("key6", Pair.create("Fiona", 39));
> put("key7", Pair.create("Francis", 41));
> put("key8", Pair.create("Charley", 21));
> put("key9", Pair.create("Amely", 40));
> }};
> loadData(part2, true);
> Map> part3 = new HashMap Pair>()
> {{
> put("key10", Pair.create("Eddie", 42));
> put("key11", Pair.create("Oswaldo", 35));
> put("key12", Pair.create("Susana", 35));
> put("key13", Pair.create("Alivia", 42));
> put("key14", Pair.create("Demario", 28));
> }};
> ColumnFamilyStore store = loadData(part3, true);
> final ByteBuffer firstName = 
> UTF8Type.instance.decompose("first_name");
> Set rows;
> rows = getIndexed(store, 10,
>   buildExpression(firstName, Operator.LIKE_SUFFIX, 
> UTF8Type.instance.decompose("a")));
> Map> concat = new HashMap<>();
> concat.putAll(part1);
> concat.putAll(part2);
> concat.putAll(part3);
> Set keys = new HashSet<>();
> concat.forEach((k, v) -> {
> int idx = v.left.indexOf('a');
> if (idx > 0)
> {
> keys.add(k);
> }
> });
> System.out.println("keys = " + keys);
> String[] arr = new String[keys.size()];
> keys.toArray(arr);
> Assert.assertTrue(rows.toString(), Arrays.equals(arr, 
> rows.toArray(new String[rows.size()])));
> }
> {code}
> Expected results are (emphasis mine)
> {code}
> [key14, key13, key0, key12, key6, key3, key4, key7, key8, key11]
> "key14"  "Dem_a_rio" 
> "key13"  "Alivi_a_" 
> "key0"   "M_a_xie" 
> "key12"  "Sus_a_na" 
> "key6"   "Fion_a_" 
> "key3"   "Sh_a_nna" 
> "key4"   "Amiy_a_" 
> "key7"   "Fr_a_ncis"
> "key8"   "Ch_a_rley" 
> "key11"  "Osw_a_ldo" 
> {code}
> although returned are
> {code}
> [key12, key13, key3, key4, key6]
> {code}
> cc [~xedin] [~doanduyhai] [~beobal] 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12128) Unexpected amount of returned results after querying SASI index

2016-07-03 Thread Alex Petrov (JIRA)
Alex Petrov created CASSANDRA-12128:
---

 Summary: Unexpected amount of returned results after querying SASI 
index
 Key: CASSANDRA-12128
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12128
 Project: Cassandra
  Issue Type: Bug
Reporter: Alex Petrov


I'm working on [CASSANDRA-11990] and one of the unit tests started failing 
after the changes I've made. However, it seems that the results returned on 
{{trunk}} are incorrect. 

One can reproduce it with the following code:

{code}
@Test
public void testCrossSSTableQueries2() throws Exception
{
Map> part1 = new HashMap>()
{{
put("key0", Pair.create("Maxie", 43));
put("key1", Pair.create("Chelsie", 33));
put("key2", Pair.create("Josephine", 43));
put("key3", Pair.create("Shanna", 27));
put("key4", Pair.create("Amiya", 36));
}};

loadData(part1, true); // first sstable

Map> part2 = new HashMap>()
{{
put("key5", Pair.create("Americo", 20));
put("key6", Pair.create("Fiona", 39));
put("key7", Pair.create("Francis", 41));
put("key8", Pair.create("Charley", 21));
put("key9", Pair.create("Amely", 40));
}};

loadData(part2, true);

Map> part3 = new HashMap>()
{{
put("key10", Pair.create("Eddie", 42));
put("key11", Pair.create("Oswaldo", 35));
put("key12", Pair.create("Susana", 35));
put("key13", Pair.create("Alivia", 42));
put("key14", Pair.create("Demario", 28));
}};

ColumnFamilyStore store = loadData(part3, true);

final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");

Set rows;

rows = getIndexed(store, 10,
  buildExpression(firstName, Operator.LIKE_SUFFIX, 
UTF8Type.instance.decompose("a")));

Map> concat = new HashMap<>();
concat.putAll(part1);
concat.putAll(part2);
concat.putAll(part3);
Set keys = new HashSet<>();
concat.forEach((k, v) -> {
int idx = v.left.indexOf('a');
if (idx > 0)
{
keys.add(k);
}
});
System.out.println("keys = " + keys);
String[] arr = new String[keys.size()];
keys.toArray(arr);
Assert.assertTrue(rows.toString(), Arrays.equals(arr, rows.toArray(new 
String[rows.size()])));
}
{code}

Expected results are (emphasis mine)
{code}
[key14, key13, key0, key12, key6, key3, key4, key7, key8, key11]

"key14"  "Dem_a_rio" 
"key13"  "Alivi_a_" 
"key0"   "M_a_xie" 
"key12"  "Sus_a_na" 
"key6"   "Fion_a_" 
"key3"   "Sh_a_nna" 
"key4"   "Amiy_a_" 
"key7"   "Fr_a_ncis"
"key8"   "Ch_a_rley" 
"key11"  "Osw_a_ldo" 
{code}

although returned are

{code}
[key12, key13, key3, key4, key6]
{code}

cc [~xedin] [~doanduyhai] [~beobal] 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11740) Nodes have wrong membership view of the cluster

2016-07-03 Thread Dikang Gu (JIRA)

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

Dikang Gu commented on CASSANDRA-11740:
---

[~jjordan], thanks, take a look at the code, if the PFS is not present, then 
the GPFS may fall back to load the DC/Rack from system.peers table. Is the 
table guaranteed to be source of truth? 

> Nodes have wrong membership view of the cluster
> ---
>
> Key: CASSANDRA-11740
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11740
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Dikang Gu
>Assignee: Joel Knighton
> Fix For: 2.2.x, 3.x
>
>
> We have a few hundreds nodes across 3 data centers, and we are doing a few 
> millions writes per second into the cluster.
> The problem we found is that there are some nodes (>10) have very wrong view 
> of the cluster.
> For example, we have 3 data centers A, B and C. On the problem nodes, in the 
> output of the 'nodetool status', it shows that ~100 nodes are not in data 
> center A, B, or C. Instead, it shows nodes are in DC1, and rack r1, which is 
> very wrong. And as a result, the node will return wrong results to client 
> requests.
> {code}
> Datacenter: DC1
> ===
> Status=Up/Down
> / State=Normal/Leaving/Joining/Moving
> – Address Load Tokens Owns Host ID Rack
> UN 2401:db00:11:6134:face:0:1:0 509.52 GB 256 ? 
> e24656ac-c3b2-4117-b933-a5b06852c993 r1
> UN 2401:db00:11:b218:face:0:5:0 510.01 GB 256 ? 
> 53da2104-b1b5-4fa5-a3dd-52c7557149f9 r1
> UN 2401:db00:2130:5133:face:0:4d:0 459.75 GB 256 ? 
> ef8311f0-f6b8-491c-904d-baa925cdd7c2 r1
> {code}
> We are using GossipingPropertyFileSnitch.
> Thanks



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)